diff --git a/.gitignore b/.gitignore index 99b32a6770e3d..a9dad34e54f6c 100644 --- a/.gitignore +++ b/.gitignore @@ -7,9 +7,12 @@ lib_managed/ src_managed/ project/boot/ project/plugins/project/ +patch-process/* .idea .svn .classpath +/.metadata +/.recommenders *~ *# .#* @@ -20,4 +23,17 @@ TAGS .settings .gradle kafka.ipr -kafka.iws \ No newline at end of file +kafka.iws +.vagrant +Vagrantfile.local +/logs + +config/server-* +config/zookeeper-* +core/data/* +gradle/wrapper/* + +results +tests/results +.ducktape +tests/.ducktape 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/CONTRIBUTING.md b/CONTRIBUTING.md new file mode 100644 index 0000000000000..e9a4d25e87ed5 --- /dev/null +++ b/CONTRIBUTING.md @@ -0,0 +1,11 @@ +## Contributing to Kafka + +*Before opening a pull request*, review the [Contributing](http://kafka.apache.org/contributing.html) and [Contributing Code Changes](https://cwiki.apache.org/confluence/display/KAFKA/Contributing+Code+Changes) pages. + +It lists steps that are required before creating a PR. + +When you contribute code, you affirm that the contribution is your original work and that you +license the work to the project under the project's open source license. Whether or not you +state this explicitly, by submitting any copyrighted material via pull request, email, or +other means you agree to license the material under the project's open source license and +warrant that you have the legal authority to do so. diff --git a/README.md b/README.md index 9aca90664b2a8..d714df79612ca 100644 --- a/README.md +++ b/README.md @@ -34,6 +34,11 @@ Follow instuctions in http://kafka.apache.org/documentation.html#quickstart ### 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 @@ -49,11 +54,11 @@ The release file can be found inside ./core/build/distributions/. ### Cleaning the build ### ./gradlew clean -### Running a task on a particular version of Scala (either 2.9.1, 2.9.2, 2.10.1 or 2.11) ### +### Running a task on a particular version of Scala (either 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 + ./gradlew -PscalaVersion=2.11.7 jar + ./gradlew -PscalaVersion=2.11.7 test + ./gradlew -PscalaVersion=2.11.7 releaseTarGz ### Running a task for a specific project ### This is for 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples' and 'clients' @@ -88,11 +93,27 @@ Please note for this to work you should create/update `~/.gradle/gradle.properti signing.password= signing.secretKeyRingFile= +### 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 the test jar ### ./gradlew testJar ### Determining how transitive dependencies are added ### ./gradlew core:dependencies --configuration runtime + +### Running checkstyle on the java code ### + ./gradlew checkstyleMain checkstyleTest + +### Limit the number of processes for each task ### + ./gradlew -Dorg.gradle.project.maxParallelForks=1 test + +This will most commonly be useful for automated builds where the full resources of the host running the build and tests +may not be dedicated to Kafka's build. + +### Running in Vagrant ### + +See [vagrant/README.md](vagrant/README.md). ### Contribution ### diff --git a/Vagrantfile b/Vagrantfile new file mode 100644 index 0000000000000..31b99b4b4f0eb --- /dev/null +++ b/Vagrantfile @@ -0,0 +1,209 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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 +base_box = "ubuntu/trusty64" + +# EC2 +ec2_access_key = ENV['AWS_ACCESS_KEY'] +ec2_secret_key = ENV['AWS_SECRET_KEY'] +ec2_session_token = ENV['AWS_SESSION_TOKEN'] +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 + +# This is a horrible hack to work around bad interactions between +# vagrant-hostmanager and vagrant-aws/vagrant's implementation. Hostmanager +# wants to update the /etc/hosts entries, but tries to do so even on nodes that +# aren't up (e.g. even when all nodes are stopped and you run vagrant +# destroy). Because of the way the underlying code in vagrant works, it still +# tries to communicate with the node and has to wait for a very long +# timeout. This modifies the update to check for hosts that are not created or +# stopped, skipping the update in that case since it's impossible to update +# nodes in that state. +Object.const_get("VagrantPlugins").const_get("HostManager").const_get("HostsFile").class_eval do + alias_method :old_update_guest, :update_guest + def update_guest(machine) + state_id = machine.state.id + return if state_id == :not_created || state_id == :stopped + old_update_guest(machine) + end +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 = base_box + + 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") + override.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. + override.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" + + cached_addresses = {} + # Use a custom resolver that SSH's into the machine and finds the IP address + # directly. This lets us get at the private IP address directly, avoiding + # some issues with using the default IP resolver, which uses the public IP + # address. + override.hostmanager.ip_resolver = proc do |vm, resolving_vm| + if !cached_addresses.has_key?(vm.name) + state_id = vm.state.id + if state_id != :not_created && state_id != :stopped && vm.communicate.ready? + vm.communicate.execute("/sbin/ifconfig eth0 | grep 'inet addr' | tail -n 1 | egrep -o '[0-9\.]+' | head -n 1 2>&1") do |type, contents| + cached_addresses[vm.name] = contents.split("\n").first[/(\d+\.\d+\.\d+\.\d+)/, 1] + end + else + cached_addresses[vm.name] = nil + end + end + cached_addresses[vm.name] + end + + 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.session_token = ec2_session_token + 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 + override.vm.synced_folder ".", "/vagrant", type: "rsync", :rsync_excludes => ['.git', 'core/data/', 'logs/', 'system_test/', 'tests/results/', 'results/'] + 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/copycat-distributed.sh b/bin/copycat-distributed.sh new file mode 100755 index 0000000000000..4d62300e00d13 --- /dev/null +++ b/bin/copycat-distributed.sh @@ -0,0 +1,23 @@ +#!/bin/sh +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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) + +if [ "x$KAFKA_LOG4J_OPTS" = "x" ]; then + export KAFKA_LOG4J_OPTS="-Dlog4j.configuration=file:$base_dir/../config/copycat-log4j.properties" +fi + +exec $(dirname $0)/kafka-run-class.sh org.apache.kafka.copycat.cli.CopycatDistributed "$@" diff --git a/bin/copycat-standalone.sh b/bin/copycat-standalone.sh new file mode 100755 index 0000000000000..b219f8aac1b37 --- /dev/null +++ b/bin/copycat-standalone.sh @@ -0,0 +1,23 @@ +#!/bin/sh +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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) + +if [ "x$KAFKA_LOG4J_OPTS" = "x" ]; then + export KAFKA_LOG4J_OPTS="-Dlog4j.configuration=file:$base_dir/../config/copycat-log4j.properties" +fi + +exec $(dirname $0)/kafka-run-class.sh org.apache.kafka.copycat.cli.CopycatStandalone "$@" diff --git a/bin/kafka-acls.sh b/bin/kafka-acls.sh new file mode 100755 index 0000000000000..fd0fb675633c9 --- /dev/null +++ b/bin/kafka-acls.sh @@ -0,0 +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. + +exec $(dirname $0)/kafka-run-class.sh kafka.admin.AclCommand $@ diff --git a/bin/kafka-configs.sh b/bin/kafka-configs.sh new file mode 100755 index 0000000000000..417eaf57e559b --- /dev/null +++ b/bin/kafka-configs.sh @@ -0,0 +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. + +exec $(dirname $0)/kafka-run-class.sh kafka.admin.ConfigCommand $@ diff --git a/bin/kafka-consumer-groups.sh b/bin/kafka-consumer-groups.sh new file mode 100755 index 0000000000000..f4786dbda01f2 --- /dev/null +++ b/bin/kafka-consumer-groups.sh @@ -0,0 +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. + +exec $(dirname $0)/kafka-run-class.sh kafka.admin.ConsumerGroupCommand $@ diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index 36c742b67a725..d98533d9a8d32 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -22,17 +22,8 @@ fi base_dir=$(dirname $0)/.. -# create logs directory -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.10.1 + SCALA_VERSION=2.10.5 fi if [ -z "$SCALA_BINARY_VERSION" ]; then @@ -65,6 +56,29 @@ do CLASSPATH=$CLASSPATH:$file done +for file in $base_dir/stream/build/libs/kafka-streams*.jar; +do + CLASSPATH=$CLASSPATH:$file +done + +for file in $base_dir/tools/build/libs/kafka-tools*.jar; +do + CLASSPATH=$CLASSPATH:$file +done + +for file in $base_dir/tools/build/dependant-libs-${SCALA_VERSION}*/*.jar; +do + CLASSPATH=$CLASSPATH:$file +done + +for cc_pkg in "api" "runtime" "file" "json" +do + for file in $base_dir/copycat/${cc_pkg}/build/libs/copycat-${cc_pkg}*.jar $base_dir/copycat/${cc_pkg}/build/dependant-libs/*.jar; + do + CLASSPATH=$CLASSPATH:$file + done +done + # classpath addition for release for file in $base_dir/libs/*.jar; do @@ -86,9 +100,20 @@ if [ $JMX_PORT ]; then KAFKA_JMX_OPTS="$KAFKA_JMX_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT " fi +# Log directory to use +if [ "x$LOG_DIR" = "x" ]; then + LOG_DIR="$base_dir/logs" +fi + # Log4j settings if [ -z "$KAFKA_LOG4J_OPTS" ]; then + # Log to console. This is a tool. KAFKA_LOG4J_OPTS="-Dlog4j.configuration=file:$base_dir/config/tools-log4j.properties" +else + # create logs directory + if [ ! -d "$LOG_DIR" ]; then + mkdir -p "$LOG_DIR" + fi fi KAFKA_LOG4J_OPTS="-Dkafka.logs.dir=$LOG_DIR $KAFKA_LOG4J_OPTS" @@ -112,7 +137,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:+UseG1GC -XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35 -XX:+DisableExplicitGC -Djava.awt.headless=true" fi @@ -125,7 +150,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 diff --git a/bin/kafka-server-start.sh b/bin/kafka-server-start.sh index 1737d0c1cb086..dc01d46a13f16 100755 --- a/bin/kafka-server-start.sh +++ b/bin/kafka-server-start.sh @@ -16,7 +16,7 @@ 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) diff --git a/bin/kafka-verifiable-producer.sh b/bin/kafka-verifiable-producer.sh new file mode 100755 index 0000000000000..d0aa6c5591de4 --- /dev/null +++ b/bin/kafka-verifiable-producer.sh @@ -0,0 +1,20 @@ +#!/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 [ "x$KAFKA_HEAP_OPTS" = "x" ]; then + export KAFKA_HEAP_OPTS="-Xmx512M" +fi +exec $(dirname $0)/kafka-run-class.sh org.apache.kafka.clients.tools.VerifiableProducer $@ diff --git a/bin/windows/kafka-acls.bat b/bin/windows/kafka-acls.bat new file mode 100644 index 0000000000000..7a78ae8a80462 --- /dev/null +++ b/bin/windows/kafka-acls.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.AclCommand %* diff --git a/bin/windows/kafka-run-class.bat b/bin/windows/kafka-run-class.bat index 8e9780e2eb74a..644969e68a138 100644 --- a/bin/windows/kafka-run-class.bat +++ b/bin/windows/kafka-run-class.bat @@ -28,7 +28,7 @@ popd set CLASSPATH= IF ["%SCALA_VERSION%"] EQU [""] ( - set SCALA_VERSION=2.10.1 + set SCALA_VERSION=2.10.5 ) IF ["%SCALA_BINARY_VERSION%"] EQU [""] ( @@ -109,7 +109,7 @@ IF ["%KAFKA_HEAP_OPTS%"] EQU [""] ( 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 + set KAFKA_JVM_PERFORMANCE_OPTS=-server -XX:+UseG1GC -XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35 -XX:+DisableExplicitGC -Djava.awt.headless=true ) IF ["%CLASSPATH%"] EQU [""] ( diff --git a/build.gradle b/build.gradle index 2e488a1ab0437..2b0f66be59ed5 100644 --- a/build.gradle +++ b/build.gradle @@ -4,23 +4,35 @@ // The ASF licenses this file to You under the Apache License, Version 2.0 // (the "License"); you may not use this file except in compliance with // the License. You may obtain a copy of the License at -// +// // http://www.apache.org/licenses/LICENSE-2.0 -// +// // Unless required by applicable law or agreed to in writing, software // distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT 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' +def slf4japi="org.slf4j:slf4j-api:1.7.6" +def junit='junit:junit:4.11' +def easymock='org.easymock:easymock:3.3.1' +def powermock='org.powermock:powermock-module-junit4:1.6.2' +def powermock_easymock='org.powermock:powermock-api-easymock:1.6.2' allprojects { apply plugin: 'idea' @@ -30,16 +42,40 @@ allprojects { } ext { - gradleVersion = "2.0" -} + gradleVersion = "2.4" + buildVersionFileName = "kafka-version.properties" + + userMaxForks = project.hasProperty('maxParallelForks') ? maxParallelForks.toInteger() : null -def isVerificationRequired(project) { - project.gradle.startParameter.taskNames.contains("uploadArchives") + 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('gradle/license.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/**', + 'CONTRIBUTING.md', + 'gradlew', + 'gradlew.bat', + '**/README.md', + '.reviewboardrc', + 'system_test/**' + ]) +} subprojects { apply plugin: 'java' @@ -47,32 +83,39 @@ subprojects { apply plugin: 'maven' apply plugin: 'signing' - licenseTest.onlyIf { isVerificationRequired(project) } + sourceCompatibility = 1.7 + + if (JavaVersion.current().isJava8Compatible()) { + tasks.withType(Javadoc) { + // disable the crazy super-strict doclint tool in Java 8 + //noinspection SpellCheckingInspection + options.addStringOption('Xdoclint:none', '-quiet') + } + } uploadArchives { repositories { signing { - if (isVerificationRequired(project)) { - 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' - } + 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' } } } @@ -82,6 +125,10 @@ subprojects { } } + tasks.withType(Test) { + maxParallelForks = userMaxForks ?: Runtime.runtime.availableProcessors() + } + jar { from '../LICENSE' from '../NOTICE' @@ -107,7 +154,7 @@ subprojects { archives srcJar archives javadocJar } - + plugins.withType(ScalaPlugin) { //source jar should also contain scala source: srcJar.from sourceSets.main.scala @@ -132,12 +179,12 @@ subprojects { configure(scalaCompileOptions.forkOptions) { memoryMaximumSize = '1g' - jvmArgs = ['-XX:MaxPermSize=512m'] + jvmArgs = ['-XX:MaxPermSize=512m', '-Xss2m'] } } } -for ( sv in ['2_9_1', '2_9_2', '2_10_1', '2_11'] ) { +for ( sv in ['2_10_5', '2_11_7'] ) { String svInDot = sv.replaceAll( "_", ".") tasks.create(name: "jar_core_${sv}", type: GradleBuild) { @@ -177,54 +224,53 @@ for ( sv in ['2_9_1', '2_9_2', '2_10_1', '2_11'] ) { } } -tasks.create(name: "jarAll", dependsOn: ['jar_core_2_9_1', 'jar_core_2_9_2', 'jar_core_2_10_1', 'jar_core_2_11', 'clients:jar', 'examples:jar', 'contrib:hadoop-consumer:jar', 'contrib:hadoop-producer:jar']) { -} +def copycatPkgs = ['copycat:api', 'copycat:runtime', 'copycat:json', 'copycat:file'] +def pkgs = ['clients', 'examples', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'log4j-appender', 'tools', 'streams'] + copycatPkgs -tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_9_1', 'srcJar_2_9_2', 'srcJar_2_10_1', 'srcJar_2_11', 'clients:srcJar', 'examples:srcJar', 'contrib:hadoop-consumer:srcJar', 'contrib:hadoop-producer:srcJar']) { } +tasks.create(name: "jarCopycat", dependsOn: copycatPkgs.collect { it + ":jar" }) {} +tasks.create(name: "jarAll", dependsOn: ['jar_core_2_10_5', 'jar_core_2_11_7'] + pkgs.collect { it + ":jar" }) { } -tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_9_1', 'docsJar_2_9_2', 'docsJar_2_10_1', 'docsJar_2_11', 'clients:docsJar', 'examples:docsJar', 'contrib:hadoop-consumer:docsJar', 'contrib:hadoop-producer:docsJar']) { } +tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_10_5', 'srcJar_2_11_7'] + pkgs.collect { it + ":srcJar" }) { } -tasks.create(name: "testAll", dependsOn: ['test_core_2_9_1', 'test_core_2_9_2', 'test_core_2_10_1', 'test_core_2_11', 'clients:test']) { -} +tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_10_5', 'docsJar_2_11_7'] + pkgs.collect { it + ":docsJar" }) { } -tasks.create(name: "releaseTarGzAll", dependsOn: ['releaseTarGz_2_9_1', 'releaseTarGz_2_9_2', 'releaseTarGz_2_10_1', 'releaseTarGz_2_11']) { -} +tasks.create(name: "testCopycat", dependsOn: copycatPkgs.collect { it + ":test" }) {} +tasks.create(name: "testAll", dependsOn: ['test_core_2_10_5', 'test_core_2_11_7'] + pkgs.collect { it + ":test" }) { } -tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_9_1', 'uploadCoreArchives_2_9_2', 'uploadCoreArchives_2_10_1', 'uploadCoreArchives_2_11', 'clients:uploadArchives', 'examples:uploadArchives', 'contrib:hadoop-consumer:uploadArchives', 'contrib:hadoop-producer:uploadArchives']) { +tasks.create(name: "releaseTarGzAll", dependsOn: ['releaseTarGz_2_10_5', 'releaseTarGz_2_11_7']) { } +tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_10_5', 'uploadCoreArchives_2_11_7'] + pkgs.collect { it + ":uploadArchives" }) { } + project(':core') { println "Building project 'core' with Scala version $scalaVersion" apply plugin: 'scala' archivesBaseName = "kafka_${baseScalaVersion}" - def (major, minor, trivial) = scalaVersion.tokenize('.') - dependencies { compile project(':clients') + compile project(':log4j-appender') compile "org.scala-lang:scala-library:$scalaVersion" compile 'org.apache.zookeeper:zookeeper:3.4.6' - compile 'com.101tec:zkclient:0.3' + compile 'com.101tec:zkclient:0.6' compile 'com.yammer.metrics:metrics-core:2.2.0' compile 'net.sf.jopt-simple:jopt-simple:3.2' + 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 'org.easymock:easymock:3.0' + testCompile "$junit" + testCompile "$easymock" testCompile 'org.objenesis:objenesis:1.2' + testCompile 'org.bouncycastle:bcpkix-jdk15on:1.52' + testCompile "org.scalatest:scalatest_$baseScalaVersion:2.2.5" testCompile project(':clients') - if (scalaVersion.startsWith('2.10')) { - testCompile 'org.scalatest:scalatest_2.10:1.9.1' - } else if (scalaVersion.startsWith('2.11')) { - compile 'org.scala-lang.modules:scala-xml_2.11:1.0.2' - compile 'org.scala-lang.modules:scala-parser-combinators_2.11:1.0.2' - testCompile "org.scalatest:scalatest_2.11:2.2.0" - } else { - testCompile "org.scalatest:scalatest_$scalaVersion:1.8" - } + testCompile project(':clients').sourceSets.test.output testRuntime "$slf4jlog4j" - zinc 'com.typesafe.zinc:zinc:0.3.1' + zinc 'com.typesafe.zinc:zinc:0.3.7' } configurations { @@ -235,6 +281,7 @@ project(':core') { compile.exclude module: 'jmxri' compile.exclude module: 'jmxtools' compile.exclude module: 'mail' + compile.exclude module: 'netty' } tasks.create(name: "copyDependantLibs", type: Copy) { @@ -247,21 +294,35 @@ project(':core') { into "$buildDir/dependant-libs-${scalaVersion}" } - tasks.create(name: "releaseTarGz", dependsOn: configurations.archives.artifacts, type: Tar) { + task siteDocsTar(type: Tar) { + classifier = 'site-docs' + compression = Compression.GZIP + from project.file("../docs") + into 'site-docs' + } + + tasks.create(name: "releaseTarGz", dependsOn: configurations.archives.artifacts, type: Tar) { into "kafka_${baseScalaVersion}-${version}" - compression = Compression.GZIP + compression = Compression.GZIP from(project.file("../bin")) { into "bin/" } from(project.file("../config")) { into "config/" } from '../LICENSE' from '../NOTICE' from(configurations.runtime) { into("libs/") } from(configurations.archives.artifacts.files) { into("libs/") } + from(project.siteDocsTar) { into("site-docs/") } } jar { dependsOn 'copyDependantLibs' } + jar.manifest { + attributes( + 'Version': "${version}" + ) + } + task testJar(type: Jar) { classifier = 'test' from sourceSets.test.output @@ -338,17 +399,223 @@ project(':examples') { } project(':clients') { + apply plugin: 'checkstyle' archivesBaseName = "kafka-clients" dependencies { - compile "org.slf4j:slf4j-api:1.7.6" - compile 'org.xerial.snappy:snappy-java:1.1.1.3' + compile "$slf4japi" + compile 'org.xerial.snappy:snappy-java:1.1.1.7' compile 'net.jpountz.lz4:lz4:1.2.0' - testCompile 'com.novocode:junit-interface:0.9' + testCompile 'org.bouncycastle:bcpkix-jdk15on:1.52' + testCompile "$junit" testRuntime "$slf4jlog4j" } + task determineCommitId { + ext.commitId = "unknown" + def takeFromHash = 16 + if (file("../.git/HEAD").exists()) { + def headRef = file("../.git/HEAD").text + if (headRef.contains('ref: ')) { + headRef = headRef.replaceAll('ref: ', '').trim() + commitId = file("../.git/$headRef").text.trim().take(takeFromHash) + } else { + commitId = headRef.trim().take(takeFromHash) + } + } else { + commitId + } + } + + task createVersionFile(dependsOn: determineCommitId) { + ext.receiptFile = file("$buildDir/kafka/$buildVersionFileName") + outputs.file receiptFile + outputs.upToDateWhen { false } + doLast { + def data = [ + commitId: determineCommitId.commitId, + version: version, + ] + + receiptFile.parentFile.mkdirs() + def content = data.entrySet().collect { "$it.key=$it.value" }.sort().join("\n") + receiptFile.setText(content, "ISO-8859-1") + } + } + + jar { + dependsOn createVersionFile + from("$buildDir") { + include "kafka/$buildVersionFileName" + } + } + + clean.doFirst { + delete "$buildDir/kafka/" + } + + task testJar(type: Jar) { + classifier = 'test' + from sourceSets.test.output + } + + test { + testLogging { + events "passed", "skipped", "failed" + 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(':tools') { + apply plugin: 'checkstyle' + archivesBaseName = "kafka-tools" + + dependencies { + compile project(':clients') + compile 'net.sourceforge.argparse4j:argparse4j:0.5.0' + compile 'com.fasterxml.jackson.core:jackson-databind:2.5.4' + compile "$slf4jlog4j" + + testCompile "$junit" + 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/tools/*" + } + + tasks.create(name: "copyDependantLibs", type: Copy) { + from (configurations.testRuntime) { + include('slf4j-log4j12*') + } + from (configurations.runtime) { + exclude('kafka-clients*') + } + into "$buildDir/dependant-libs-${scalaVersion}" + } + + jar { + dependsOn 'copyDependantLibs' + } + + artifacts { + archives testJar + } + + configurations { + archives.extendsFrom (testCompile) + } + + checkstyle { + configFile = new File(rootDir, "checkstyle/checkstyle.xml") + } + test.dependsOn('checkstyleMain', 'checkstyleTest') +} + +project(':streams') { + apply plugin: 'checkstyle' + archivesBaseName = "kafka-streams" + + dependencies { + compile project(':clients') + compile "$slf4jlog4j" + compile 'org.rocksdb:rocksdbjni:3.10.1' + + testCompile "$junit" + 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/streams/*" + } + + tasks.create(name: "copyDependantLibs", type: Copy) { + from (configurations.testRuntime) { + include('slf4j-log4j12*') + } + from (configurations.runtime) { + exclude('kafka-clients*') + } + into "$buildDir/dependant-libs-${scalaVersion}" + } + + jar { + dependsOn 'copyDependantLibs' + } + + 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" + testCompile project(path: ':clients', configuration: 'archives') + } + task testJar(type: Jar) { classifier = 'test' from sourceSets.test.output @@ -360,4 +627,205 @@ project(':clients') { exceptionFormat = 'full' } } + + javadoc { + include "**/org/apache/kafka/log4jappender/*" + } + + checkstyle { + configFile = new File(rootDir, "checkstyle/checkstyle.xml") + } + test.dependsOn('checkstyleMain', 'checkstyleTest') +} + +project(':copycat:api') { + apply plugin: 'checkstyle' + archivesBaseName = "copycat-api" + + dependencies { + compile "$slf4japi" + compile project(':clients') + + testCompile "$junit" + testRuntime "$slf4jlog4j" + } + + task testJar(type: Jar) { + classifier = 'test' + from sourceSets.test.output + } + + test { + testLogging { + events "passed", "skipped", "failed" + exceptionFormat = 'full' + } + } + + javadoc { + include "**/org/apache/kafka/copycat/*" + } + + artifacts { + archives testJar + } + + configurations { + archives.extendsFrom (testCompile) + } + + checkstyle { + configFile = new File(rootDir, "checkstyle/checkstyle.xml") + } + test.dependsOn('checkstyleMain', 'checkstyleTest') +} + +project(':copycat:json') { + apply plugin: 'checkstyle' + archivesBaseName = "copycat-json" + + dependencies { + compile project(':copycat:api') + compile "$slf4japi" + compile 'com.fasterxml.jackson.core:jackson-databind:2.5.4' + + testCompile "$junit" + testCompile "$easymock" + testCompile "$powermock" + testCompile "$powermock_easymock" + testRuntime "$slf4jlog4j" + } + + task testJar(type: Jar) { + classifier = 'test' + from sourceSets.test.output + } + + test { + testLogging { + events "passed", "skipped", "failed" + exceptionFormat = 'full' + } + } + + javadoc { + include "**/org/apache/kafka/copycat/*" + } + + artifacts { + archives testJar + } + + configurations { + archives.extendsFrom(testCompile) + } + + checkstyle { + configFile = new File(rootDir, "checkstyle/checkstyle.xml") + } + test.dependsOn('checkstyleMain', 'checkstyleTest') + + tasks.create(name: "copyDependantLibs", type: Copy) { + from (configurations.runtime) { + exclude('kafka-clients*') + exclude('copycat-*') + } + into "$buildDir/dependant-libs" + } + + jar { + dependsOn copyDependantLibs + } +} + +project(':copycat:runtime') { + apply plugin: 'checkstyle' + archivesBaseName = "copycat-runtime" + + dependencies { + compile project(':copycat:api') + compile project(':clients') + compile "$slf4japi" + + testCompile "$junit" + testCompile "$easymock" + testCompile "$powermock" + testCompile "$powermock_easymock" + testRuntime "$slf4jlog4j" + testRuntime project(":copycat:json") + } + + task testJar(type: Jar) { + classifier = 'test' + from sourceSets.test.output + } + + test { + testLogging { + events "passed", "skipped", "failed" + exceptionFormat = 'full' + } + } + + javadoc { + include "**/org/apache/kafka/copycat/*" + } + + artifacts { + archives testJar + } + + configurations { + archives.extendsFrom(testCompile) + } + + checkstyle { + configFile = new File(rootDir, "checkstyle/checkstyle.xml") + } + test.dependsOn('checkstyleMain', 'checkstyleTest') +} + +project(':copycat:file') { + apply plugin: 'checkstyle' + archivesBaseName = "copycat-file" + + dependencies { + compile project(':copycat:api') + compile "$slf4japi" + + testCompile "$junit" + testCompile "$easymock" + testCompile "$powermock" + testCompile "$powermock_easymock" + testRuntime "$slf4jlog4j" + } + + task testJar(type: Jar) { + classifier = 'test' + from sourceSets.test.output + } + + test { + testLogging { + events "passed", "skipped", "failed" + exceptionFormat = 'full' + } + } + + javadoc { + include "**/org/apache/kafka/copycat/*" + } + + artifacts { + archives testJar + } + + configurations { + archives.extendsFrom(testCompile) + } + + 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..cd0207cc341ff --- /dev/null +++ b/checkstyle/checkstyle.xml @@ -0,0 +1,87 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml new file mode 100644 index 0000000000000..247f556083330 --- /dev/null +++ b/checkstyle/import-control.xml @@ -0,0 +1,195 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java b/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java index d32c319d8ee4c..117b0bf22ec92 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java @@ -19,27 +19,50 @@ */ public final class ClientRequest { - private final long createdMs; + private final long createdTimeMs; private final boolean expectResponse; private final RequestSend request; - private final Object attachment; + private final RequestCompletionHandler callback; + private final boolean isInitiatedByNetworkClient; + private long sendTimeMs; /** - * @param createdMs The unix timestamp in milliseconds for the time at which this request was created. + * @param createdTimeMs 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 attachment Associated data with 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, Object attachment) { - this.createdMs = createdMs; - this.attachment = attachment; + public ClientRequest(long createdTimeMs, boolean expectResponse, RequestSend request, + RequestCompletionHandler callback) { + this(createdTimeMs, expectResponse, request, callback, false); + } + + /** + * @param createdTimeMs 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) + * @param isInitiatedByNetworkClient Is request initiated by network client, if yes, its + * response will be consumed by network client + */ + public ClientRequest(long createdTimeMs, boolean expectResponse, RequestSend request, + RequestCompletionHandler callback, boolean isInitiatedByNetworkClient) { + this.createdTimeMs = createdTimeMs; + this.callback = callback; this.request = request; this.expectResponse = expectResponse; + this.isInitiatedByNetworkClient = isInitiatedByNetworkClient; } @Override public String toString() { - return "ClientRequest(expectResponse=" + expectResponse + ", payload=" + attachment + ", request=" + request + ")"; + return "ClientRequest(expectResponse=" + expectResponse + + ", callback=" + callback + + ", request=" + request + + (isInitiatedByNetworkClient ? ", isInitiatedByNetworkClient" : "") + + ", createdTimeMs=" + createdTimeMs + + ", sendTimeMs=" + sendTimeMs + + ")"; } public boolean expectResponse() { @@ -50,12 +73,27 @@ public RequestSend request() { return request; } - public Object attachment() { - return attachment; + public boolean hasCallback() { + return callback != null; } - public long createdTime() { - return createdMs; + public RequestCompletionHandler callback() { + return callback; } -} \ No newline at end of file + public long createdTimeMs() { + return createdTimeMs; + } + + public boolean isInitiatedByNetworkClient() { + return isInitiatedByNetworkClient; + } + + public long sendTimeMs() { + return sendTimeMs; + } + + public void setSendTimeMs(long sendTimeMs) { + this.sendTimeMs = sendTimeMs; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java b/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java index 14ef69afabc4b..3b6f9556296ec 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java @@ -20,27 +20,27 @@ */ public class ClientResponse { - private final long received; + private final long receivedTimeMs; 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 receivedTimeMs 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) { + public ClientResponse(ClientRequest request, long receivedTimeMs, boolean disconnected, Struct responseBody) { super(); - this.received = received; + this.receivedTimeMs = receivedTimeMs; this.disconnected = disconnected; this.request = request; this.responseBody = responseBody; } - public long receivedTime() { - return received; + public long receivedTimeMs() { + return receivedTimeMs; } public boolean wasDisconnected() { @@ -60,12 +60,12 @@ public boolean hasResponse() { } public long requestLatencyMs() { - return receivedTime() - this.request.createdTime(); + return receivedTimeMs() - this.request.createdTimeMs(); } @Override public String toString() { - return "ClientResponse(received=" + received + + return "ClientResponse(receivedTimeMs=" + receivedTimeMs + ", disconnected=" + disconnected + ", request=" + diff --git a/clients/src/main/java/org/apache/kafka/common/utils/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java similarity index 53% rename from clients/src/main/java/org/apache/kafka/common/utils/ClientUtils.java rename to clients/src/main/java/org/apache/kafka/clients/ClientUtils.java index b987e7f0434c6..e7514f815f9db 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -3,26 +3,36 @@ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the * License. You may obtain a copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES 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; +package org.apache.kafka.clients; +import java.io.Closeable; import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; -import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.network.ChannelBuilders; +import org.apache.kafka.common.protocol.SecurityProtocol; +import org.apache.kafka.common.network.ChannelBuilder; +import org.apache.kafka.common.security.ssl.SSLFactory; 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(); @@ -31,19 +41,42 @@ public static List parseAndValidateAddresses(List url String host = getHost(url); Integer port = getPort(url); if (host == null || port == null) - throw new ConfigException("Invalid url in " + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + ": " + url); + 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 " + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + ": " + url); + 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 " + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + ": " + url); + throw new ConfigException("Invalid port in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + ": " + url); } } } if (addresses.size() < 1) - throw new ConfigException("No bootstrap urls given in " + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG); + throw new ConfigException("No bootstrap urls given in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG); return addresses; } -} \ No newline at end of file + + 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); + } + } + } + + /** + * @param configs client/server configs + * @return configured ChannelBuilder based on the configs. + */ + public static ChannelBuilder createChannelBuilder(Map configs) { + SecurityProtocol securityProtocol = SecurityProtocol.valueOf((String) configs.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)); + if (securityProtocol != SecurityProtocol.SSL && securityProtocol != SecurityProtocol.PLAINTEXT) + throw new ConfigException("Invalid SecurityProtocol " + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG); + return ChannelBuilders.create(securityProtocol, SSLFactory.Mode.CLIENT, configs); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java index d304660f29246..6c582115c0a2e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java @@ -21,22 +21,22 @@ */ final class ClusterConnectionStates { private final long reconnectBackoffMs; - private final Map nodeState; + private final Map nodeState; public ClusterConnectionStates(long reconnectBackoffMs) { this.reconnectBackoffMs = reconnectBackoffMs; - this.nodeState = new HashMap(); + this.nodeState = new HashMap(); } /** - * Return true iff we can currently initiate a new connection to the given node. This will be the case if we are not + * 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 node The node id to check + * @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(int node, long now) { - NodeConnectionState state = nodeState.get(node); + public boolean canConnect(String id, long now) { + NodeConnectionState state = nodeState.get(id); if (state == null) return true; else @@ -45,11 +45,11 @@ public boolean canConnect(int node, long now) { /** * Return true if we are disconnected from the given node and can't re-establish a connection yet - * @param node The node to check + * @param id The connection to check * @param now The current time in ms */ - public boolean isBlackedOut(int node, long now) { - NodeConnectionState state = nodeState.get(node); + public boolean isBlackedOut(String id, long now) { + NodeConnectionState state = nodeState.get(id); if (state == null) return false; else @@ -57,57 +57,116 @@ public boolean isBlackedOut(int node, long now) { } /** - * Enter the connecting state for the given node. - * @param node The id of the node we are connecting to + * 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(int node, long now) { - nodeState.put(node, new NodeConnectionState(ConnectionState.CONNECTING, now)); + public void connecting(String id, long now) { + nodeState.put(id, new NodeConnectionState(ConnectionState.CONNECTING, now)); } /** - * Return true iff we have a connection to the give node - * @param node The id of the node to check + * Return true iff a specific connection is connected + * @param id The id of the connection to check */ - public boolean isConnected(int node) { - NodeConnectionState state = nodeState.get(node); + 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 to the given node - * @param node The id of the node + * Return true iff we are in the process of connecting + * @param id The id of the connection */ - public boolean isConnecting(int node) { - NodeConnectionState state = nodeState.get(node); + public boolean isConnecting(String id) { + NodeConnectionState state = nodeState.get(id); return state != null && state.state == ConnectionState.CONNECTING; } /** - * Enter the connected state for the given node - * @param node The node we have connected to + * Enter the connected state for the given connection + * @param id The connection identifier */ - public void connected(int node) { - nodeState(node).state = ConnectionState.CONNECTED; + public void connected(String id) { + NodeConnectionState nodeState = nodeState(id); + nodeState.state = ConnectionState.CONNECTED; } /** * Enter the disconnected state for the given node - * @param node The node we have disconnected from + * @param id The connection we have disconnected */ - public void disconnected(int node) { - nodeState(node).state = ConnectionState.DISCONNECTED; + public void disconnected(String id) { + NodeConnectionState nodeState = nodeState(id); + nodeState.state = ConnectionState.DISCONNECTED; } /** - * Get the state of our connection to the given state - * @param node The id of the node + * Remove the given node from the tracked connection states. The main difference between this and `disconnected` + * is the impact on `connectionDelay`: it will be 0 after this call whereas `reconnectBackoffMs` will be taken + * into account after `disconnected` is called. + * + * @param id The connection to remove + */ + public void remove(String id) { + nodeState.remove(id); + } + + /** + * Get the state of a given connection + * @param id The id of the connection * @return The state of our connection */ - private NodeConnectionState nodeState(int node) { - NodeConnectionState state = this.nodeState.get(node); + 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 node " + node); + throw new IllegalStateException("No entry found for connection " + id); return state; } -} \ No newline at end of file + + /** + * 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 + ")"; + } + } +} 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..48e4919f68a6b --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.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.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 SECURITY_PROTOCOL_CONFIG = "security.protocol"; + public static final String SECURITY_PROTOCOL_DOC = "Protocol used to communicate with brokers. Currently only PLAINTEXT and SSL are supported."; + public static final String DEFAULT_SECURITY_PROTOCOL = "PLAINTEXT"; + + 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."; + + public static final String REQUEST_TIMEOUT_MS_CONFIG = "request.timeout.ms"; + public static final String REQUEST_TIMEOUT_MS_DOC = "The configuration controls the maximum amount of time the client will wait " + + "for the response of a request. If the response is not received before the timeout " + + "elapses the client will resend the request if necessary or fail the request if " + + "retries are exhausted."; +} diff --git a/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java b/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java index ab7e3220f9b76..3867f8ecd5925 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java @@ -15,6 +15,6 @@ /** * The states of a node connection */ -enum ConnectionState { +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 index 936487b16e7ac..8de19ee4f11f5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java +++ b/clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java @@ -16,6 +16,8 @@ import java.util.Collections; import java.util.Deque; import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; import java.util.Map; /** @@ -24,19 +26,19 @@ final class InFlightRequests { private final int maxInFlightRequestsPerConnection; - private final Map> requests = new HashMap>(); + private final Map> requests = new HashMap>(); public InFlightRequests(int maxInFlightRequestsPerConnection) { this.maxInFlightRequestsPerConnection = maxInFlightRequestsPerConnection; } /** - * Add the given request to the queue for the node it was directed to + * 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(); + reqs = new ArrayDeque<>(); this.requests.put(request.request().destination(), reqs); } reqs.addFirst(request); @@ -45,7 +47,7 @@ public void add(ClientRequest request) { /** * Get the request queue for the given node */ - private Deque requestQueue(int 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."); @@ -55,7 +57,7 @@ private Deque requestQueue(int node) { /** * Get the oldest request (the one that that will be completed next) for the given node */ - public ClientRequest completeNext(int node) { + public ClientRequest completeNext(String node) { return requestQueue(node).pollLast(); } @@ -63,7 +65,7 @@ public ClientRequest completeNext(int node) { * 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(int node) { + public ClientRequest lastSent(String node) { return requestQueue(node).peekFirst(); } @@ -72,7 +74,7 @@ public ClientRequest lastSent(int node) { * @param node The node the request was sent to * @return The request */ - public ClientRequest completeLastSent(int node) { + public ClientRequest completeLastSent(String node) { return requestQueue(node).pollFirst(); } @@ -82,7 +84,7 @@ public ClientRequest completeLastSent(int 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) { + public boolean canSendMore(String node) { Deque queue = requests.get(node); return queue == null || queue.isEmpty() || (queue.peekFirst().request().completed() && queue.size() < this.maxInFlightRequestsPerConnection); @@ -93,7 +95,7 @@ public boolean canSendMore(int node) { * @param node The node * @return The request count. */ - public int inFlightRequestCount(int node) { + public int inFlightRequestCount(String node) { Deque queue = requests.get(node); return queue == null ? 0 : queue.size(); } @@ -114,7 +116,7 @@ public int inFlightRequestCount() { * @param node The node * @return All the in-flight requests for that node that have been removed */ - public Iterable clearAll(int node) { + public Iterable clearAll(String node) { Deque reqs = requests.get(node); if (reqs == null) { return Collections.emptyList(); @@ -123,4 +125,25 @@ public Iterable clearAll(int node) { } } -} \ No newline at end of file + /** + * Returns a list of nodes with pending inflight request, that need to be timed out + * + * @param now current time in milliseconds + * @param requestTimeout max time to wait for the request to be completed + * @return list of nodes + */ + public List getNodesWithTimedOutRequests(long now, int requestTimeout) { + List nodeIds = new LinkedList(); + for (String nodeId : requests.keySet()) { + if (inFlightRequestCount(nodeId) > 0) { + ClientRequest request = requests.get(nodeId).peekLast(); + long timeSinceSend = now - request.sendTimeMs(); + if (timeSinceSend > requestTimeout) { + nodeIds.add(nodeId); + } + } + } + + return nodeIds; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java index 29658d4a15f11..8c6e39a1a1c3b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java @@ -12,6 +12,7 @@ */ package org.apache.kafka.clients; +import java.io.Closeable; import java.util.List; import org.apache.kafka.common.Node; @@ -21,11 +22,12 @@ /** * The interface for {@link NetworkClient} */ -public interface KafkaClient { +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 */ @@ -34,6 +36,7 @@ public interface KafkaClient { /** * 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. @@ -41,19 +44,57 @@ public interface KafkaClient { public boolean ready(Node node, long now); /** - * Initiate the sending of the given requests and return any completed responses. Requests can only be sent on ready + * 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 requests The requests to send - * @param timeout The maximum amount of time to wait for responses in ms + * + * @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 + * @param now The current timestamp + */ + public void send(ClientRequest request, long now); + + /** + * Do actual reads and writes from sockets. + * + * @param timeout The maximum amount of time to wait for responses in ms, must be non-negative. The implementation + * is free to use a lower value if appropriate (common reasons for this are a lower request or + * metadata update timeout) * @param now The current time in ms * @throws IllegalStateException If a request is sent to an unready node */ - public List poll(List requests, long timeout, long now); + public List poll(long timeout, long now); + + /** + * Closes the connection to a particular node (if there is one). + * + * @param nodeId The id of the node + */ + public void close(String nodeId); /** * 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. */ @@ -64,20 +105,32 @@ public interface KafkaClient { */ 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 + * Generate a request header for the given API key + * + * @param key The api key + * @param version The api version + * @return A request header with the appropriate client id and correlation id */ - public void wakeup(); + public RequestHeader nextRequestHeader(ApiKeys key, short version); /** - * Close the client and disconnect from all nodes + * Wake up the client if it is currently blocked waiting for I/O */ - public void close(); + public void wakeup(); -} \ No newline at end of file +} diff --git a/clients/src/main/java/org/apache/kafka/clients/ManualMetadataUpdater.java b/clients/src/main/java/org/apache/kafka/clients/ManualMetadataUpdater.java new file mode 100644 index 0000000000000..efbe6646c2433 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/ManualMetadataUpdater.java @@ -0,0 +1,76 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package org.apache.kafka.clients; + +import org.apache.kafka.common.Node; +import org.apache.kafka.common.protocol.types.Struct; + +import java.util.ArrayList; +import java.util.List; + +/** + * A simple implementation of `MetadataUpdater` that returns the cluster nodes set via the constructor or via + * `setNodes`. + * + * This is useful in cases where automatic metadata updates are not required. An example is controller/broker + * communication. + * + * This class is not thread-safe! + */ +public class ManualMetadataUpdater implements MetadataUpdater { + + private List nodes; + + public ManualMetadataUpdater() { + this(new ArrayList(0)); + } + + public ManualMetadataUpdater(List nodes) { + this.nodes = nodes; + } + + public void setNodes(List nodes) { + this.nodes = nodes; + } + + @Override + public List fetchNodes() { + return new ArrayList<>(nodes); + } + + @Override + public boolean isUpdateDue(long now) { + return false; + } + + @Override + public long maybeUpdate(long now) { + return Long.MAX_VALUE; + } + + @Override + public boolean maybeHandleDisconnection(ClientRequest request) { + return false; + } + + @Override + public boolean maybeHandleCompletedReceive(ClientRequest request, long now, Struct body) { + return false; + } + + @Override + public void requestUpdate() { + // Do nothing + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java similarity index 51% rename from clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java rename to clients/src/main/java/org/apache/kafka/clients/Metadata.java index 1d30f9edd9533..f2fca12e09d52 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java @@ -10,12 +10,17 @@ * an "AS IS" BASIS, WITHOUT WARRANTIES 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; +package org.apache.kafka.clients; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.HashSet; +import java.util.List; import java.util.Set; - import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.errors.TimeoutException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -25,7 +30,7 @@ *

* 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 + * 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 { @@ -36,9 +41,12 @@ public final class Metadata { private final long metadataExpireMs; private int version; private long lastRefreshMs; + private long lastSuccessfulRefreshMs; private Cluster cluster; private boolean needUpdate; private final Set topics; + private final List listeners; + private boolean needMetadataForAllTopics; /** * Create a metadata instance with reasonable defaults @@ -57,10 +65,13 @@ 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(); + this.listeners = new ArrayList<>(); + this.needMetadataForAllTopics = false; } /** @@ -78,12 +89,12 @@ public synchronized void add(String 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 + * 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.lastRefreshMs + this.metadataExpireMs - nowMs, 0); + long timeToExpire = needUpdate ? 0 : Math.max(this.lastSuccessfulRefreshMs + this.metadataExpireMs - nowMs, 0); long timeToAllowUpdate = this.lastRefreshMs + this.refreshBackoffMs - nowMs; return Math.max(timeToExpire, timeToAllowUpdate); } @@ -99,14 +110,15 @@ public synchronized int requestUpdate() { /** * Wait for metadata update until the current version is larger than the last version we know of */ - public synchronized void awaitUpdate(int lastVerison, long maxWaitMs) { + 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 <= lastVerison) { - try { + while (this.version <= lastVersion) { + if (remainingWaitMs != 0) wait(remainingWaitMs); - } catch (InterruptedException e) { /* this is fine */ - } long elapsed = System.currentTimeMillis() - begin; if (elapsed >= maxWaitMs) throw new TimeoutException("Failed to update metadata after " + maxWaitMs + " ms."); @@ -114,6 +126,17 @@ public synchronized void awaitUpdate(int lastVerison, long maxWaitMs) { } } + /** + * Replace the current set of topics maintained to the one provided + * @param topics + */ + public synchronized void setTopics(Collection topics) { + if (!this.topics.containsAll(topics)) + requestUpdate(); + this.topics.clear(); + this.topics.addAll(topics); + } + /** * Get the list of topics we are currently maintaining metadata for */ @@ -121,23 +144,54 @@ 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; + + for (Listener listener: listeners) + listener.onMetadataUpdate(cluster); + + // Do this after notifying listeners as subscribed topics' list can be changed by listeners + this.cluster = this.needMetadataForAllTopics ? getClusterForCurrentTopics(cluster) : cluster; + notifyAll(); log.debug("Updated cluster metadata version {} to {}", this.version, this.cluster); } /** - * The last time metadata was updated. + * 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 lastUpdate() { - return this.lastRefreshMs; + public synchronized long lastSuccessfulUpdate() { + return this.lastSuccessfulRefreshMs; } /** @@ -146,4 +200,52 @@ public synchronized long lastUpdate() { public long refreshBackoff() { return refreshBackoffMs; } + + /** + * Set state to indicate if metadata for all topics in Kafka cluster is required or not. + * @param needMetadaForAllTopics boolean indicating need for metadata of all topics in cluster. + */ + public void needMetadataForAllTopics(boolean needMetadaForAllTopics) { + this.needMetadataForAllTopics = needMetadaForAllTopics; + } + + /** + * Get whether metadata for all topics is needed or not + */ + public boolean needMetadataForAllTopics() { + return this.needMetadataForAllTopics; + } + + /** + * Add a Metadata listener that gets notified of metadata updates + */ + public void addListener(Listener listener) { + this.listeners.add(listener); + } + + /** + * Stop notifying the listener of metadata updates + */ + public void removeListener(Listener listener) { + this.listeners.remove(listener); + } + + /** + * MetadataUpdate Listener + */ + public interface Listener { + void onMetadataUpdate(Cluster cluster); + } + + private Cluster getClusterForCurrentTopics(Cluster cluster) { + Collection partitionInfos = new ArrayList<>(); + List nodes = Collections.emptyList(); + if (cluster != null) { + for (String topic : this.topics) { + partitionInfos.addAll(cluster.partitionsForTopic(topic)); + } + nodes = cluster.nodes(); + } + return new Cluster(nodes, partitionInfos); + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java b/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java new file mode 100644 index 0000000000000..4669a6846ab18 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java @@ -0,0 +1,72 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES 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.Node; +import org.apache.kafka.common.protocol.types.Struct; + +import java.util.List; + +/** + * The interface used by `NetworkClient` to request cluster metadata info to be updated and to retrieve the cluster nodes + * from such metadata. This is an internal class. + *

+ * This class is not thread-safe! + */ +interface MetadataUpdater { + + /** + * Gets the current cluster info without blocking. + */ + List fetchNodes(); + + /** + * Returns true if an update to the cluster metadata info is due. + */ + boolean isUpdateDue(long now); + + /** + * Starts a cluster metadata update if needed and possible. Returns the time until the metadata update (which would + * be 0 if an update has been started as a result of this call). + * + * If the implementation relies on `NetworkClient` to send requests, the completed receive will be passed to + * `maybeHandleCompletedReceive`. + * + * The semantics of `needed` and `possible` are implementation-dependent and may take into account a number of + * factors like node availability, how long since the last metadata update, etc. + */ + long maybeUpdate(long now); + + /** + * If `request` is a metadata request, handles it and return `true`. Otherwise, returns `false`. + * + * This provides a mechanism for the `MetadataUpdater` implementation to use the NetworkClient instance for its own + * requests with special handling for disconnections of such requests. + */ + boolean maybeHandleDisconnection(ClientRequest request); + + /** + * If `request` is a metadata request, handles it and returns `true`. Otherwise, returns `false`. + * + * This provides a mechanism for the `MetadataUpdater` implementation to use the NetworkClient instance for its own + * requests with special handling for completed receives of such requests. + */ + boolean maybeHandleCompletedReceive(ClientRequest request, long now, Struct body); + + /** + * Schedules an update of the current cluster metadata info. A subsequent call to `maybeUpdate` would trigger the + * start of the update if possible (see `maybeUpdate` for more information). + */ + void requestUpdate(); +} diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index eea270abb16f4..6f39ac9046e54 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -3,9 +3,9 @@ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the * License. You may obtain a copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the * specific language governing permissions and limitations under the License. @@ -15,16 +15,16 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; import java.util.Random; import java.util.Set; -import org.apache.kafka.clients.producer.internals.Metadata; 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.NetworkSend; 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; @@ -49,9 +49,8 @@ public class NetworkClient implements KafkaClient { /* the selector used to perform network i/o */ private final Selectable selector; - - /* the current cluster metadata */ - private final Metadata metadata; + + private final MetadataUpdater metadataUpdater; /* the state of each node's connection */ private final ClusterConnectionStates connectionStates; @@ -74,11 +73,8 @@ public class NetworkClient implements KafkaClient { /* 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; + /* max time in ms for the producer to wait for acknowledgement from server*/ + private final int requestTimeoutMs; public NetworkClient(Selectable selector, Metadata metadata, @@ -86,9 +82,45 @@ public NetworkClient(Selectable selector, int maxInFlightRequestsPerConnection, long reconnectBackoffMs, int socketSendBuffer, - int socketReceiveBuffer) { + int socketReceiveBuffer, + int requestTimeoutMs) { + this(null, metadata, selector, clientId, maxInFlightRequestsPerConnection, + reconnectBackoffMs, socketSendBuffer, socketReceiveBuffer, requestTimeoutMs); + } + + public NetworkClient(Selectable selector, + MetadataUpdater metadataUpdater, + String clientId, + int maxInFlightRequestsPerConnection, + long reconnectBackoffMs, + int socketSendBuffer, + int socketReceiveBuffer, + int requestTimeoutMs) { + this(metadataUpdater, null, selector, clientId, maxInFlightRequestsPerConnection, reconnectBackoffMs, + socketSendBuffer, socketReceiveBuffer, requestTimeoutMs); + } + + private NetworkClient(MetadataUpdater metadataUpdater, + Metadata metadata, + Selectable selector, + String clientId, + int maxInFlightRequestsPerConnection, + long reconnectBackoffMs, + int socketSendBuffer, + int socketReceiveBuffer, int requestTimeoutMs) { + + /* It would be better if we could pass `DefaultMetadataUpdater` from the public constructor, but it's not + * possible because `DefaultMetadataUpdater` is an inner class and it can only be instantiated after the + * super constructor is invoked. + */ + if (metadataUpdater == null) { + if (metadata == null) + throw new IllegalArgumentException("`metadata` must not be null"); + this.metadataUpdater = new DefaultMetadataUpdater(metadata); + } else { + this.metadataUpdater = metadataUpdater; + } this.selector = selector; - this.metadata = metadata; this.clientId = clientId; this.inFlightRequests = new InFlightRequests(maxInFlightRequestsPerConnection); this.connectionStates = new ClusterConnectionStates(reconnectBackoffMs); @@ -96,12 +128,12 @@ public NetworkClient(Selectable selector, this.socketReceiveBuffer = socketReceiveBuffer; this.correlation = 0; this.nodeIndexOffset = new Random().nextInt(Integer.MAX_VALUE); - this.metadataFetchInProgress = false; - this.lastNoNodeAvailableMs = 0; + this.requestTimeoutMs = requestTimeoutMs; } /** * 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 @@ -111,80 +143,142 @@ public boolean ready(Node node, long now) { if (isReady(node, now)) return true; - if (connectionStates.canConnect(node.id(), now)) + 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; } + /** + * Closes the connection to a particular node (if there is one). + * + * @param nodeId The id of the node + */ + @Override + public void close(String nodeId) { + selector.close(nodeId); + for (ClientRequest request : inFlightRequests.clearAll(nodeId)) + metadataUpdater.maybeHandleDisconnection(request); + connectionStates.remove(nodeId); + } + + /** + * 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 given node id + * + * @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) { - int nodeId = node.id(); - 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); + // if we need to update our metadata now declare all requests unready to make metadata requests first + // priority + return !metadataUpdater.isUpdateDue(now) && canSendRequest(node.idString()); } /** - * Are we connected and ready and able to send more requests to the given node? + * Are we connected and ready and able to send more requests to the given connection? + * * @param node The node */ - private boolean isSendable(int node) { - return connectionStates.isConnected(node) && inFlightRequests.canSendMore(node); + private boolean canSendRequest(String node) { + return connectionStates.isConnected(node) && selector.isChannelReady(node) && inFlightRequests.canSendMore(node); } /** - * Initiate the given requests and check for any new responses, waiting up to the specified time. Requests can only - * be sent for ready nodes. - * @param requests The requests to initiate - * @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 + * Return the state of the connection to the given node + * + * @param node The node to check + * @return The connection state */ - @Override - public List poll(List requests, long timeout, long now) { - List sends = new ArrayList(); - - for (int i = 0; i < requests.size(); i++) { - ClientRequest request = requests.get(i); - int nodeId = request.request().destination(); - if (!isSendable(nodeId)) - throw new IllegalStateException("Attempt to send a request to node " + nodeId + " which is not ready."); + public ConnectionState connectionState(String node) { + return connectionStates.connectionState(node); + } - this.inFlightRequests.add(request); - sends.add(request.request()); - } + /** + * Queue up the given request for sending. Requests can only be sent out to ready nodes. + * + * @param request The request + * @param now The current timestamp + */ + @Override + public void send(ClientRequest request, long now) { + String nodeId = request.request().destination(); + if (!canSendRequest(nodeId)) + throw new IllegalStateException("Attempt to send a request to node " + nodeId + " which is not ready."); + doSend(request, now); + } - // should we update our metadata? - long timeToNextMetadataUpdate = metadata.timeToNextUpdate(now); - long timeToNextReconnectAttempt = this.lastNoNodeAvailableMs + metadata.refreshBackoff() - now; - // if there is no node available to connect, back off refreshing metadata - long metadataTimeout = Math.max(timeToNextMetadataUpdate, timeToNextReconnectAttempt); - if (!this.metadataFetchInProgress && metadataTimeout == 0) - maybeUpdateMetadata(sends, now); + private void doSend(ClientRequest request, long now) { + request.setSendTimeMs(now); + this.inFlightRequests.add(request); + selector.send(request.request()); + } - // do the I/O + /** + * 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, + * must be non-negative. The actual timeout will be the minimum of timeout, request timeout and + * metadata timeout + * @param now The current time in milliseconds + * @return The list of responses received + */ + @Override + public List poll(long timeout, long now) { + long metadataTimeout = metadataUpdater.maybeUpdate(now); try { - this.selector.poll(Math.min(timeout, metadataTimeout), sends); + this.selector.poll(Utils.min(timeout, metadataTimeout, requestTimeoutMs)); } catch (IOException e) { log.error("Unexpected error during I/O in producer network thread", e); } - List responses = new ArrayList(); + // process completed actions + List responses = new ArrayList<>(); handleCompletedSends(responses, now); handleCompletedReceives(responses, now); handleDisconnections(responses, now); handleConnections(); + handleTimedOutRequests(responses, now); + + // 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; } @@ -197,8 +291,17 @@ 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 */ @@ -207,6 +310,18 @@ public RequestHeader nextRequestHeader(ApiKeys key) { return new RequestHeader(key.id, clientId, correlation++); } + /** + * Generate a request header for the given API key and version + * + * @param key The api key + * @param version The api version + * @return A request header with the appropriate client id and correlation id + */ + @Override + public RequestHeader nextRequestHeader(ApiKeys key, short version) { + return new RequestHeader(key.id, version, clientId, correlation++); + } + /** * Interrupt the client if it is blocked waiting on I/O. */ @@ -228,37 +343,76 @@ public void close() { * 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. */ + @Override public Node leastLoadedNode(long now) { - List nodes = this.metadata.fetch().nodes(); + List nodes = this.metadataUpdater.fetchNodes(); 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.id()); - if (currInflight == 0 && this.connectionStates.isConnected(node.id())) { + 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.id(), now) && currInflight < inflight) { + } 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; } + /** + * Post process disconnection of a node + * + * @param responses The list of responses to update + * @param nodeId Id of the node to be disconnected + * @param now The current time + */ + private void processDisconnection(List responses, String nodeId, long now) { + connectionStates.disconnected(nodeId); + for (ClientRequest request : this.inFlightRequests.clearAll(nodeId)) { + log.trace("Cancelled request {} due to node {} being disconnected", request, nodeId); + if (!metadataUpdater.maybeHandleDisconnection(request)) + responses.add(new ClientResponse(request, now, true, null)); + } + } + + /** + * Iterate over all the inflight requests and expire any requests that have exceeded the configured the requestTimeout. + * The connection to the node associated with the request will be terminated and will be treated as a disconnection. + * + * @param responses The list of responses to update + * @param now The current time + */ + private void handleTimedOutRequests(List responses, long now) { + List nodeIds = this.inFlightRequests.getNodesWithTimedOutRequests(now, this.requestTimeoutMs); + for (String nodeId : nodeIds) { + // close connection to the node + this.selector.close(nodeId); + log.debug("Disconnecting from node {} due to request timeout.", nodeId); + processDisconnection(responses, nodeId, now); + } + + // we disconnected, so we should probably refresh our metadata + if (nodeIds.size() > 0) + metadataUpdater.requestUpdate(); + } + /** * 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 (NetworkSend send : this.selector.completedSends()) { + for (Send send : this.selector.completedSends()) { ClientRequest request = this.inFlightRequests.lastSent(send.destination()); if (!request.expectResponse()) { this.inFlightRequests.completeLastSent(send.destination()); @@ -269,68 +423,46 @@ private void handleCompletedSends(List responses, long now) { /** * 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()) { - int source = receive.source(); + 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 + if (!metadataUpdater.maybeHandleCompletedReceive(req, now, body)) 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(); - // 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()); - } - /** * 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 (int node : this.selector.disconnected()) { - connectionStates.disconnected(node); + for (String node : this.selector.disconnected()) { 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)); - } + processDisconnection(responses, node, now); } // 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(); + metadataUpdater.requestUpdate(); } /** * Record any newly completed connections */ private void handleConnections() { - for (Integer id : this.selector.connected()) { - log.debug("Completed connection to node {}", id); - this.connectionStates.connected(id); + for (String node : this.selector.connected()) { + log.debug("Completed connection to node {}", node); + this.connectionStates.connected(node); } } @@ -339,63 +471,165 @@ private void handleConnections() { */ 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, int 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(List sends, long now) { - 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; - } - - log.debug("Trying to send metadata request to node {}", node.id()); - if (connectionStates.isConnected(node.id()) && inFlightRequests.canSendMore(node.id())) { - Set topics = metadata.topics(); - this.metadataFetchInProgress = true; - ClientRequest metadataRequest = metadataRequest(now, node.id(), topics); - log.debug("Sending metadata request {} to node {}", metadataRequest, node.id()); - sends.add(metadataRequest.request()); - this.inFlightRequests.add(metadataRequest); - } else if (connectionStates.canConnect(node.id(), now)) { - // we don't have a connection to this node right now, make one - log.debug("Init connection to node {} for sending metadata request in the next iteration", node.id()); - initiateConnect(node, now); - } + throw new IllegalStateException("Correlation id for response (" + responseHeader.correlationId() + + ") does not match request (" + requestHeader.correlationId() + ")"); } /** * 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()); - selector.connect(node.id(), new InetSocketAddress(node.host(), node.port()), this.socketSendBuffer, this.socketReceiveBuffer); - this.connectionStates.connecting(node.id(), now); + 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(node.id()); + connectionStates.disconnected(nodeConnectionId); /* maybe the problem is our metadata, update it */ - metadata.requestUpdate(); + metadataUpdater.requestUpdate(); log.debug("Error connecting to node {} at {}:{}:", node.id(), node.host(), node.port(), e); } } + class DefaultMetadataUpdater implements MetadataUpdater { + + /* the current cluster metadata */ + private final Metadata metadata; + + /* 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; + + DefaultMetadataUpdater(Metadata metadata) { + this.metadata = metadata; + this.metadataFetchInProgress = false; + this.lastNoNodeAvailableMs = 0; + } + + @Override + public List fetchNodes() { + return metadata.fetch().nodes(); + } + + @Override + public boolean isUpdateDue(long now) { + return !this.metadataFetchInProgress && this.metadata.timeToNextUpdate(now) == 0; + } + + @Override + public long maybeUpdate(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) { + // Beware that the behavior of this method and the computation of timeouts for poll() are + // highly dependent on the behavior of leastLoadedNode. + Node node = leastLoadedNode(now); + maybeUpdate(now, node); + } + + return metadataTimeout; + } + + @Override + public boolean maybeHandleDisconnection(ClientRequest request) { + ApiKeys requestKey = ApiKeys.forId(request.request().header().apiKey()); + + if (requestKey == ApiKeys.METADATA) { + metadataFetchInProgress = false; + return true; + } + + return false; + } + + @Override + public boolean maybeHandleCompletedReceive(ClientRequest req, long now, Struct body) { + short apiKey = req.request().header().apiKey(); + if (apiKey == ApiKeys.METADATA.id && req.isInitiatedByNetworkClient()) { + handleResponse(req.request().header(), body, now); + return true; + } + return false; + } + + @Override + public void requestUpdate() { + this.metadata.requestUpdate(); + } + + private void handleResponse(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); + } + } + + /** + * Create a metadata request for the given topics + */ + private ClientRequest request(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, true); + } + + /** + * Add a metadata request to the list of sends if we can make one + */ + private void maybeUpdate(long now, Node node) { + 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 (canSendRequest(nodeConnectionId)) { + Set topics = metadata.needMetadataForAllTopics() ? new HashSet() : metadata.topics(); + this.metadataFetchInProgress = true; + ClientRequest metadataRequest = request(now, nodeConnectionId, topics); + log.debug("Sending metadata request {} to node {}", metadataRequest, node.id()); + doSend(metadataRequest, now); + } 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; + } + } + + } + } diff --git a/clients/src/main/java/org/apache/kafka/clients/NodeConnectionState.java b/clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java similarity index 66% rename from clients/src/main/java/org/apache/kafka/clients/NodeConnectionState.java rename to clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java index 752a979ea0b8b..6fee4e45986d2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NodeConnectionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java @@ -13,19 +13,11 @@ package org.apache.kafka.clients; /** - * The state of our connection to a node + * 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. */ -final class NodeConnectionState { +public interface RequestCompletionHandler { - ConnectionState state; - long lastConnectAttemptMs; + public void onComplete(ClientResponse response); - 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/consumer/Consumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java index 227f5646ee708..a3d8776083032 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java @@ -9,117 +9,151 @@ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES 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.Collection; +import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.regex.Pattern; import org.apache.kafka.common.Metric; +import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.annotation.InterfaceStability; /** * @see KafkaConsumer * @see MockConsumer */ -public interface Consumer extends Closeable { +@InterfaceStability.Unstable +public interface Consumer extends Closeable { + + /** + * @see KafkaConsumer#assignment() + */ + public Set assignment(); /** - * Incrementally subscribe to the given list of topics. This API is mutually exclusive to - * {@link #subscribe(TopicPartition...) subscribe(partitions)} - * @param topics A variable list of topics that the consumer subscribes to - */ - public void subscribe(String...topics); + * @see KafkaConsumer#subscription() + */ + public Set subscription(); /** - * Incrementally subscribes to a specific topic and partition. This API is mutually exclusive to - * {@link #subscribe(String...) subscribe(topics)} - * @param partitions Partitions to subscribe to - */ - public void subscribe(TopicPartition... partitions); + * @see KafkaConsumer#subscribe(List) + */ + public void subscribe(List topics); /** - * Unsubscribe from the specific topics. Messages for this topic will not be returned from the next {@link #poll(long) poll()} - * onwards. This should be used in conjunction with {@link #subscribe(String...) subscribe(topics)}. It is an error to - * unsubscribe from a topic that was never subscribed to using {@link #subscribe(String...) subscribe(topics)} - * @param topics Topics to unsubscribe from + * @see KafkaConsumer#subscribe(List, ConsumerRebalanceListener) */ - public void unsubscribe(String... topics); + public void subscribe(List topics, ConsumerRebalanceListener callback); /** - * Unsubscribe from the specific topic partitions. Messages for these partitions will not be returned from the next - * {@link #poll(long) poll()} onwards. This should be used in conjunction with - * {@link #subscribe(TopicPartition...) subscribe(topic, partitions)}. It is an error to - * unsubscribe from a partition that was never subscribed to using {@link #subscribe(TopicPartition...) subscribe(partitions)} - * @param partitions Partitions to unsubscribe from + * @see KafkaConsumer#assign(List) */ - public void unsubscribe(TopicPartition... partitions); - + public void assign(List partitions); + + /** + * @see KafkaConsumer#subscribe(Pattern, ConsumerRebalanceListener) + */ + public void subscribe(Pattern pattern, ConsumerRebalanceListener callback); + /** - * Fetches data for the subscribed list of topics and partitions - * @param timeout The time, in milliseconds, spent waiting in poll if data is not available. If 0, waits indefinitely. Must not be negative - * @return Map of topic to records for the subscribed topics and partitions as soon as data is available for a topic partition. Availability - * of data is controlled by {@link ConsumerConfig#FETCH_MIN_BYTES_CONFIG} and {@link ConsumerConfig#FETCH_MAX_WAIT_MS_CONFIG}. - * If no data is available for timeout ms, returns an empty list + * @see KafkaConsumer#unsubscribe() */ - public Map poll(long timeout); + public void unsubscribe(); /** - * Commits offsets returned on the last {@link #poll(long) poll()} for the subscribed list of topics and partitions. - * @param sync If true, the commit should block until the consumer receives an acknowledgment - * @return An {@link OffsetMetadata} object that contains the partition, offset and a corresponding error code. Returns null - * if the sync flag is set to false + * @see KafkaConsumer#poll(long) */ - public OffsetMetadata commit(boolean sync); + public ConsumerRecords poll(long timeout); /** - * Commits the specified offsets for the specified list of topics and partitions to Kafka. - * @param offsets The map of offsets to commit for the given topic partitions - * @param sync If true, commit will block until the consumer receives an acknowledgment - * @return An {@link OffsetMetadata} object that contains the partition, offset and a corresponding error code. Returns null - * if the sync flag is set to false. + * @see KafkaConsumer#commitSync() */ - public OffsetMetadata commit(Map offsets, boolean sync); - + public void commitSync(); + /** - * Overrides the fetch positions that the consumer will use on the next fetch request. If the consumer subscribes to a list of topics - * using {@link #subscribe(String...) subscribe(topics)}, an exception will be thrown if the specified topic partition is not owned by - * the consumer. - * @param offsets The map of fetch positions per topic and partition + * @see KafkaConsumer#commitSync(Map) */ - public void seek(Map offsets); + public void commitSync(Map offsets); /** - * Returns the fetch position of the next message for the specified topic partition to be used on the next {@link #poll(long) poll()} - * @param partitions Partitions for which the fetch position will be returned - * @return The position from which data will be fetched for the specified partition on the next {@link #poll(long) poll()} + * @see KafkaConsumer#commitAsync() */ - public Map position(Collection partitions); - + public void commitAsync(); + /** - * Fetches the last committed offsets for the input list of partitions - * @param partitions The list of partitions to return the last committed offset for - * @return The list of offsets for the specified list of partitions + * @see KafkaConsumer#commitAsync(OffsetCommitCallback) */ - public Map committed(Collection partitions); - + public void commitAsync(OffsetCommitCallback callback); + /** - * Fetches offsets before a certain timestamp - * @param timestamp The unix timestamp. Value -1 indicates earliest available timestamp. Value -2 indicates latest available timestamp. - * @param partitions The list of partitions for which the offsets are returned - * @return The offsets for messages that were written to the server before the specified timestamp. + * @see KafkaConsumer#commitAsync(Map, OffsetCommitCallback) */ - public Map offsetsBeforeTime(long timestamp, Collection partitions); - + public void commitAsync(Map offsets, OffsetCommitCallback callback); + + /** + * @see KafkaConsumer#seek(TopicPartition, long) + */ + public void seek(TopicPartition partition, long offset); + + /** + * @see KafkaConsumer#seekToBeginning(TopicPartition...) + */ + public void seekToBeginning(TopicPartition... partitions); + /** - * Return a map of metrics maintained by the consumer + * @see KafkaConsumer#seekToEnd(TopicPartition...) */ - public Map metrics(); + public void seekToEnd(TopicPartition... partitions); /** - * Close this consumer + * @see KafkaConsumer#position(TopicPartition) + */ + public long position(TopicPartition partition); + + /** + * @see KafkaConsumer#committed(TopicPartition) + */ + public OffsetAndMetadata committed(TopicPartition partition); + + /** + * @see KafkaConsumer#metrics() + */ + public Map metrics(); + + /** + * @see KafkaConsumer#partitionsFor(String) + */ + public List partitionsFor(String topic); + + /** + * @see KafkaConsumer#listTopics() + */ + public Map> listTopics(); + + /** + * @see KafkaConsumer#pause(TopicPartition...) + */ + public void pause(TopicPartition... partitions); + + /** + * @see KafkaConsumer#resume(TopicPartition...) + */ + public void resume(TopicPartition... partitions); + + /** + * @see KafkaConsumer#close() */ public void close(); + /** + * @see KafkaConsumer#wakeup() + */ + public void wakeup(); + } 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 index 46efc0c8483ac..bcc143ce6689e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -3,185 +3,350 @@ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the * License. You may obtain a copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES 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.apache.kafka.common.config.ConfigDef.Range.atLeast; - -import java.util.Map; - +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.Deserializer; +import org.apache.kafka.common.config.SSLConfigs; + +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; + 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. + */ /** - * The identifier of the group this consumer belongs to. This is required if the consumer uses either the - * group management functionality by using {@link Consumer#subscribe(String...) subscribe(topics)}. This is also required - * if the consumer uses the default Kafka based offset management strategy. + * 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."; + /** - * The timeout after which, if the {@link Consumer#poll(long) poll(timeout)} is not invoked, the consumer is - * marked dead and a rebalance operation is triggered for the group identified by {@link #GROUP_ID_CONFIG}. Relevant - * if the consumer uses the group management functionality by invoking {@link Consumer#subscribe(String...) subscribe(topics)} + * session.timeout.ms */ - public static final String SESSION_TIMEOUT_MS = "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."; /** - * The number of times a consumer sends a heartbeat to the co-ordinator broker within a {@link #SESSION_TIMEOUT_MS} time window. - * This frequency affects the latency of a rebalance operation since the co-ordinator broker notifies a consumer of a rebalance - * in the heartbeat response. Relevant if the consumer uses the group management functionality by invoking - * {@link Consumer#subscribe(String...) subscribe(topics)} + * heartbeat.interval.ms */ - public static final String HEARTBEAT_FREQUENCY = "heartbeat.frequency"; + public static final String HEARTBEAT_INTERVAL_MS_CONFIG = "heartbeat.interval.ms"; + private static final String HEARTBEAT_INTERVAL_MS_DOC = "The expected time between heartbeats to the consumer coordinator when using Kafka's group management facilities. Heartbeats are used to ensure that the consumer's session stays active and to facilitate rebalancing when new consumers join or leave the group. The value must be set lower than session.timeout.ms, but typically should be set no higher than 1/3 of that value. It can be adjusted even lower to control the expected time for normal rebalances."; /** - * 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). + * bootstrap.servers */ - public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers"; + public static final String BOOTSTRAP_SERVERS_CONFIG = CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; /** - * If true, periodically commit to Kafka the offsets of messages already returned by the consumer. This committed - * offset will be used when the process fails as the position from which the consumption will begin. + * 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."; + /** - * 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.commit.interval.ms */ - public static final String PARTITION_ASSIGNMENT_STRATEGY = "partition.assignment.strategy"; - + 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."; + /** - * The frequency in milliseconds that the consumer offsets are committed to Kafka. Relevant if {@link #ENABLE_AUTO_COMMIT_CONFIG} - * is turned on. + * partition.assignment.strategy */ - public static final String AUTO_COMMIT_INTERVAL_MS_CONFIG = "auto.commit.interval.ms"; - + 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"; + /** - * What to do when there is no initial offset in Kafka or 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 - *
  • disable: throw exception to the consumer if no previous offset is found for the consumer's group - *
  • anything else: throw exception to the consumer. - *
+ * 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):
  • earliest: automatically reset the offset to the earliest offset
  • latest: automatically reset the offset to the latest 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.
"; + /** - * 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. + * 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."; + /** - * The maximum amount of time the server will block before answering the fetch request if there isn't sufficient - * data to immediately satisfy {@link #FETCH_MIN_BYTES_CONFIG}. This should be less than or equal to the timeout used in - * {@link KafkaConsumer#poll(long) poll(timeout)} + * 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; + /** - * The maximum amount of time to block waiting to fetch metadata about a topic the first time a record is received - * from that topic. The consumer will throw a TimeoutException if it could not successfully fetch metadata within - * this timeout. + * max.partition.fetch.bytes */ - public static final String METADATA_FETCH_TIMEOUT_CONFIG = "metadata.fetch.timeout.ms"; + 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; /** - * The total memory used by the consumer to buffer records received from the server. This config is meant to control - * the consumer's memory usage, so it is the size of the global fetch buffer that will be shared across all partitions. + * client.id */ - public static final String TOTAL_BUFFER_MEMORY_CONFIG = "total.memory.bytes"; + public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG; /** - * The minimum amount of memory that should be used to fetch at least one message for a partition. This puts a lower - * bound on the consumer's memory utilization when there is at least one message for a partition available on the server. - * 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. + * reconnect.backoff.ms */ - public static final String FETCH_BUFFER_CONFIG = "fetch.buffer.bytes"; - + public static final String RECONNECT_BACKOFF_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MS_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. + * retry.backoff.ms */ - public static final String CLIENT_ID_CONFIG = "client.id"; + public static final String RETRY_BACKOFF_MS_CONFIG = CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG; /** - * The size of the TCP send buffer to use when fetching data + * metrics.sample.window.ms */ - public static final String SOCKET_RECEIVE_BUFFER_CONFIG = "socket.receive.buffer.bytes"; + public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG; /** - * 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. + * metrics.num.samples */ - public static final String RECONNECT_BACKOFF_MS_CONFIG = "reconnect.backoff.ms"; + 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; + + /** + * 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"; + public static final String KEY_DESERIALIZER_CLASS_DOC = "Deserializer class for key that implements the Deserializer interface."; - /** metrics.sample.window.ms */ - public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = "metrics.sample.window.ms"; - private static final String METRICS_SAMPLE_WINDOW_MS_DOC = "The metrics system maintains a configurable number of samples over a fixed window size. This configuration " + "controls the size of the window. For example we might maintain two samples each measured over a 30 second period. " - + "When a window expires we erase and overwrite the oldest window."; + /** value.deserializer */ + public static final String VALUE_DESERIALIZER_CLASS_CONFIG = "value.deserializer"; + public static final String VALUE_DESERIALIZER_CLASS_DOC = "Deserializer class for value that implements the Deserializer interface."; - /** metrics.num.samples */ - public static final String METRICS_NUM_SAMPLES_CONFIG = "metrics.num.samples"; - private static final String METRICS_NUM_SAMPLES_DOC = "The number of samples maintained to compute metrics."; + /** connections.max.idle.ms */ + public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG; + + /** request.timeout.ms */ + public static final String REQUEST_TIMEOUT_MS_CONFIG = CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG; + private static final String REQUEST_TIMEOUT_MS_DOC = CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC; - /** metric.reporters */ - 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. 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."; static { - /* TODO: add config docs */ - config = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, "blah blah") - .define(GROUP_ID_CONFIG, Type.STRING, Importance.HIGH, "blah blah") - .define(SESSION_TIMEOUT_MS, Type.LONG, 1000, Importance.HIGH, "blah blah") - .define(HEARTBEAT_FREQUENCY, Type.INT, 3, Importance.MEDIUM, "blah blah") - .define(PARTITION_ASSIGNMENT_STRATEGY, Type.STRING, Importance.MEDIUM, "blah blah") - .define(METADATA_FETCH_TIMEOUT_CONFIG, Type.LONG, 60 * 1000, atLeast(0), Importance.MEDIUM, "blah blah") - .define(ENABLE_AUTO_COMMIT_CONFIG, Type.BOOLEAN, true, Importance.MEDIUM, "blah blah") - .define(AUTO_COMMIT_INTERVAL_MS_CONFIG, Type.LONG, 5000, atLeast(0), Importance.LOW, "blah blah") - .define(CLIENT_ID_CONFIG, Type.STRING, "", Importance.LOW, "blah blah") - .define(TOTAL_BUFFER_MEMORY_CONFIG, Type.LONG, 32 * 1024 * 1024L, atLeast(0L), Importance.LOW, "blah blah") - .define(FETCH_BUFFER_CONFIG, Type.INT, 1 * 1024 * 1024, atLeast(0), Importance.HIGH, "blah blah") - .define(SOCKET_RECEIVE_BUFFER_CONFIG, Type.INT, 128 * 1024, atLeast(0), Importance.LOW, "blah blah") - .define(FETCH_MIN_BYTES_CONFIG, Type.LONG, 1024, atLeast(0), Importance.HIGH, "blah blah") - .define(FETCH_MAX_WAIT_MS_CONFIG, Type.LONG, 500, atLeast(0), Importance.LOW, "blah blah") - .define(RECONNECT_BACKOFF_MS_CONFIG, Type.LONG, 10L, atLeast(0L), Importance.LOW, "blah blah") - .define(AUTO_OFFSET_RESET_CONFIG, Type.STRING, "largest", Importance.MEDIUM, "blah blah") + 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(HEARTBEAT_INTERVAL_MS_CONFIG, + Type.INT, + 3000, + Importance.HIGH, + HEARTBEAT_INTERVAL_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(CHECK_CRCS_CONFIG, + Type.BOOLEAN, + true, + Importance.LOW, + CHECK_CRCS_DOC) .define(METRICS_SAMPLE_WINDOW_MS_CONFIG, Type.LONG, 30000, atLeast(0), Importance.LOW, - METRICS_SAMPLE_WINDOW_MS_DOC) - .define(METRICS_NUM_SAMPLES_CONFIG, Type.INT, 2, atLeast(1), Importance.LOW, METRICS_NUM_SAMPLES_DOC) - .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, "", Importance.LOW, METRIC_REPORTER_CLASSES_DOC); - + 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) + .define(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, Type.STRING, CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL, Importance.MEDIUM, CommonClientConfigs.SECURITY_PROTOCOL_DOC) + .define(SSLConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, Type.CLASS, SSLConfigs.DEFAULT_PRINCIPAL_BUILDER_CLASS, Importance.LOW, SSLConfigs.PRINCIPAL_BUILDER_CLASS_DOC) + .define(SSLConfigs.SSL_PROTOCOL_CONFIG, Type.STRING, SSLConfigs.DEFAULT_SSL_PROTOCOL, Importance.MEDIUM, SSLConfigs.SSL_PROTOCOL_DOC) + .define(SSLConfigs.SSL_PROVIDER_CONFIG, Type.STRING, Importance.MEDIUM, SSLConfigs.SSL_PROVIDER_DOC, false) + .define(SSLConfigs.SSL_CIPHER_SUITES_CONFIG, Type.LIST, Importance.LOW, SSLConfigs.SSL_CIPHER_SUITES_DOC, false) + .define(SSLConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Type.LIST, SSLConfigs.DEFAULT_ENABLED_PROTOCOLS, Importance.MEDIUM, SSLConfigs.SSL_ENABLED_PROTOCOLS_DOC) + .define(SSLConfigs.SSL_KEYSTORE_TYPE_CONFIG, Type.STRING, SSLConfigs.DEFAULT_SSL_KEYSTORE_TYPE, Importance.MEDIUM, SSLConfigs.SSL_KEYSTORE_TYPE_DOC) + .define(SSLConfigs.SSL_KEYSTORE_LOCATION_CONFIG, Type.STRING, Importance.HIGH, SSLConfigs.SSL_KEYSTORE_LOCATION_DOC, false) + .define(SSLConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, Type.STRING, Importance.HIGH, SSLConfigs.SSL_KEYSTORE_PASSWORD_DOC, false) + .define(SSLConfigs.SSL_KEY_PASSWORD_CONFIG, Type.STRING, Importance.HIGH, SSLConfigs.SSL_KEY_PASSWORD_DOC, false) + .define(SSLConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, Type.STRING, SSLConfigs.DEFAULT_SSL_TRUSTSTORE_TYPE, Importance.MEDIUM, SSLConfigs.SSL_TRUSTSTORE_TYPE_DOC) + .define(SSLConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, Type.STRING, SSLConfigs.DEFAULT_TRUSTSTORE_LOCATION, Importance.HIGH, SSLConfigs.SSL_TRUSTSTORE_LOCATION_DOC) + .define(SSLConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, Type.STRING, SSLConfigs.DEFAULT_TRUSTSTORE_PASSWORD, Importance.HIGH, SSLConfigs.SSL_TRUSTSTORE_PASSWORD_DOC) + .define(SSLConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, Type.STRING, SSLConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM, Importance.LOW, SSLConfigs.SSL_KEYMANAGER_ALGORITHM_DOC) + .define(SSLConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, Type.STRING, SSLConfigs.DEFAULT_SSL_TRUSTMANAGER_ALGORITHM, Importance.LOW, SSLConfigs.SSL_TRUSTMANAGER_ALGORITHM_DOC) + .define(SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, Type.STRING, Importance.LOW, SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DOC, false) + .define(REQUEST_TIMEOUT_MS_CONFIG, + Type.INT, + 40 * 1000, + atLeast(0), + Importance.MEDIUM, + REQUEST_TIMEOUT_MS_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); } - 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 deleted file mode 100644 index f026ae41ce820..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java +++ /dev/null @@ -1,50 +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.consumer; - -import java.util.Collection; - -import org.apache.kafka.common.TopicPartition; - -/** - * A callback interface that the user can implement to manage customized offsets on the start and end of - * every rebalance operation. This callback will execute in the user thread as part of the - * {@link Consumer#poll(long) poll(long)} API on every rebalance attempt. - * Default implementation of the callback will {@link Consumer#seek(java.util.Map) seek(offsets)} to the last committed offsets in the - * {@link #onPartitionsAssigned(Consumer, Collection) onPartitionsAssigned()} callback. And will commit offsets synchronously - * for the specified list of partitions to Kafka in the {@link #onPartitionsRevoked(Consumer, Collection) onPartitionsRevoked()} - * callback. - */ -public interface ConsumerRebalanceCallback { - - /** - * A callback method the user can implement to provide handling of customized offsets on completion of a successful - * rebalance operation. This method will be called after a rebalance operation completes and before the consumer - * starts fetching data. - *

- * For examples on usage of this API, see Usage Examples section of {@link KafkaConsumer KafkaConsumer} - * @param partitions The list of partitions that are assigned to the consumer after rebalance - */ - 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 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/ConsumerRebalanceListener.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceListener.java new file mode 100644 index 0000000000000..671b6f252ce28 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceListener.java @@ -0,0 +1,101 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES 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 assign 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. + * Rebalances can also be triggered by changes affecting the subscribed topics (e.g. when then number of partitions is + * administratively adjusted). + *

+ * There are many uses for this functionality. One common use is saving offsets in a custom store. By saving offsets in + * the {@link #onPartitionsRevoked(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 its 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(Collection) onPartitionsRevoked} prior to + * any process invoking {@link #onPartitionsAssigned(Collection) onPartitionsAssigned}. So if offsets or other state is saved in the + * {@link #onPartitionsRevoked(Collection) onPartitionsRevoked} call it is guaranteed to be saved by the time the process taking over that + * partition has their {@link #onPartitionsAssigned(Collection) onPartitionsAssigned} callback called to load the state. + *

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

+ * {@code
+ *   public class SaveOffsetsOnRebalance implements ConsumerRebalanceListener {
+ *       private Consumer consumer;
+ *
+ *       public SaveOffsetsOnRebalance(Consumer consumer) {
+ *           this.consumer = consumer;
+ *       }
+ *
+ *       public void onPartitionsAssigned(Collection partitions) {
+ *           // read the offsets from an external store using some custom code not described here
+ *           for(TopicPartition partition: partitions)
+ *              consumer.seek(partition, readOffsetFromExternalStore(partition));
+ *       }      
+ *       public void onPartitionsRevoked(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 ConsumerRebalanceListener { + + /** + * A callback method the user can implement to provide handling of customized offsets on completion of a successful + * partition re-assignment. This method will be called after an offset re-assignment completes and before the + * consumer starts fetching data. + *

+ * It is guaranteed that all the processes in a consumer group will execute their + * {@link #onPartitionsRevoked(Collection)} callback before any instance executes its + * {@link #onPartitionsAssigned(Collection)} callback. + * + * @param partitions The list of partitions that are now assigned to the consumer (may include partitions previously + * assigned to the consumer) + */ + public void onPartitionsAssigned(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 partitions The list of partitions that were assigned to the consumer on the last rebalance + */ + public void onPartitionsRevoked(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 index 436d8a479166e..d4668c2ddc0f1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java @@ -9,119 +9,77 @@ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES 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; - /** - * 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. + * 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 TopicPartition partition; - private final byte[] key; - private final byte[] value; +public final class ConsumerRecord { + private final String topic; + private final int partition; private final long offset; - private volatile Exception error; - - /** - * Creates a record to be received from a specified topic and partition - * - * @param topic The topic this record is received from - * @param partitionId The partition of the topic this record is received from - * @param key The key of the record, if one exists - * @param value The record contents - * @param offset The offset of this record in the corresponding Kafka partition - */ - public ConsumerRecord(String topic, int partitionId, byte[] key, byte[] value, long offset) { - this(topic, partitionId, key, value, offset, null); - } + private final K key; + private final V value; /** - * Create a record with no key - * + * Creates a record to be received from a specified topic and partition + * * @param topic The topic this record is received from - * @param partitionId The partition of the topic this record is received from - * @param value The record contents + * @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 key The key of the record, if one exists (null is allowed) + * @param value The record contents */ - public ConsumerRecord(String topic, int partitionId, byte[] value, long offset) { - this(topic, partitionId, null, value, offset); - } - - /** - * Creates a record with an error code - * @param topic The topic this record is received from - * @param partitionId The partition of the topic this record is received from - * @param error The exception corresponding to the error code returned by the server for this topic partition - */ - public ConsumerRecord(String topic, int partitionId, Exception error) { - this(topic, partitionId, null, null, -1L, error); - } - - private ConsumerRecord(String topic, int partitionId, byte[] key, byte[] value, long offset, Exception error) { + public ConsumerRecord(String topic, int partition, long offset, K key, V value) { if (topic == null) throw new IllegalArgumentException("Topic cannot be null"); - this.partition = new TopicPartition(topic, partitionId); + this.topic = topic; + this.partition = partition; + this.offset = offset; this.key = key; this.value = value; - this.offset = offset; - this.error = error; } - + /** * The topic this record is received from */ public String topic() { - return partition.topic(); + return this.topic; } /** - * The partition from which this record is received + * The partition from which this record is received */ public int partition() { - return partition.partition(); + return this.partition; } - - /** - * The TopicPartition object containing the topic and partition - */ - public TopicPartition topicAndPartition() { - return partition; - } - + /** * The key (or null if no key is specified) - * @throws Exception The exception thrown while fetching this record. */ - public byte[] key() throws Exception { - if (this.error != null) - throw this.error; + public K key() { return key; } /** * The value - * @throws Exception The exception thrown while fetching this record. */ - public byte[] value() throws Exception { - if (this.error != null) - throw this.error; + public V value() { return value; } /** * The position of this record in the corresponding Kafka partition. - * @throws Exception The exception thrown while fetching this record. */ - public long offset() throws Exception { - if (this.error != null) - throw this.error; + public long offset() { return offset; } - public Exception error() { - return this.error; + @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 index 2ecfc8aaea90a..8e6fef45dbc41 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java @@ -9,53 +9,109 @@ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES 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; -import java.util.Map.Entry; /** - * 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. + * A container that holds the list {@link ConsumerRecord} per partition for a + * particular topic. There is one {@link ConsumerRecord} list for every topic + * partition returned by a {@link Consumer#poll(long)} operation. */ -public class ConsumerRecords { +public class ConsumerRecords implements Iterable> { + public static final ConsumerRecords EMPTY = + new ConsumerRecords(Collections.EMPTY_MAP); - private final String topic; - private final Map> recordsPerPartition; - - public ConsumerRecords(String topic, Map> records) { - this.topic = topic; - this.recordsPerPartition = records; + private final Map>> records; + + public ConsumerRecords(Map>> records) { + this.records = records; } - + /** - * @param partitions The input list of partitions for a particular topic. If no partitions are - * specified, returns records for all partitions - * @return The list of {@link ConsumerRecord}s associated with the given partitions. + * Get just the records for the given partition + * + * @param partition The partition to get records for */ - public List records(int... partitions) { - List recordsToReturn = new ArrayList(); - if(partitions.length == 0) { - // return records for all partitions - for(Entry> record : recordsPerPartition.entrySet()) { - recordsToReturn.addAll(record.getValue()); - } - } else { - for(int partition : partitions) { - List recordsForThisPartition = recordsPerPartition.get(partition); - recordsToReturn.addAll(recordsForThisPartition); - } + 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 recordsToReturn; + return new ConcatenatedIterable(recs); } + @Override + public Iterator> iterator() { + return new ConcatenatedIterable(records.values()).iterator(); + } + /** - * @return The topic of all records associated with this instance + * The number of records for all topics */ - public String topic() { - return this.topic; + 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 index fe93afa24fc20..ceba667a33d26 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -3,579 +3,1220 @@ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the * License. You may obtain a copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES 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.net.InetSocketAddress; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Set; -import java.util.Map.Entry; -import java.util.concurrent.Future; - -import org.apache.kafka.clients.producer.RecordMetadata; +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.Fetcher; +import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener; +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.annotation.InterfaceStability; +import org.apache.kafka.common.config.ConfigException; 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.utils.ClientUtils; +import org.apache.kafka.common.network.ChannelBuilder; +import org.apache.kafka.common.network.Selector; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.utils.AppInfoParser; 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.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.ConcurrentModificationException; +import java.util.HashMap; +import java.util.HashSet; +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 java.util.regex.Pattern; + /** * A Kafka client that consumes records from a Kafka cluster. - *

- * The consumer is thread safe and should generally be shared among all threads for best performance. *

- * The consumer is single threaded 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. + * 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 #commitSync() 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 #commitSync() 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 its 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. Following are some examples to demonstrate the correct use of - * the available APIs. Each of the examples assumes the presence of a user implemented process() method that processes a given batch of messages - * and returns the offset of the latest processed message per partition. Note that process() is not part of the consumer API and is only used as - * a convenience method to demonstrate the different use cases of the consumer APIs. Here is a sample implementation of such a process() method. + * 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. + * *
- * {@code
- * private Map process(Map records) {
- *     Map processedOffsets = new HashMap();
- *     for(Entry recordMetadata : records.entrySet()) {
- *          List recordsPerTopic = recordMetadata.getValue().records();
- *          for(int i = 0;i < recordsPerTopic.size();i++) {
- *               ConsumerRecord record = recordsPerTopic.get(i);
- *               // process record
- *               try {
- *               	processedOffsets.put(record.topicAndpartition(), record.offset());
- *               } catch (Exception e) {
- *               	e.printStackTrace();
- *               }               
- *          }
+ *     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());
  *     }
- *     return processedOffsets; 
- * }
- * }
  * 
+ * + * Setting enable.auto.commit means that offsets are committed automatically with a frequency controlled by + * the config auto.commit.interval.ms. *

- * This example demonstrates how the consumer can be used to leverage Kafka's group management functionality for automatic consumer load - * balancing and failover. This example assumes that the offsets are stored in Kafka and are automatically committed periodically, - * as controlled by the auto.commit.interval.ms config - *

- * {@code  
- * Properties props = new Properties();
- * props.put("metadata.broker.list", "localhost:9092");
- * props.put("group.id", "test");
- * props.put("session.timeout.ms", "1000");
- * props.put("enable.auto.commit", "true");
- * props.put("auto.commit.interval.ms", "10000");
- * KafkaConsumer consumer = new KafkaConsumer(props);
- * consumer.subscribe("foo", "bar");
- * boolean isRunning = true;
- * while(isRunning) {
- *   Map records = consumer.poll(100);
- *   process(records);
- * }
- * consumer.close();
- * }
- * 
- * This example demonstrates how the consumer can be used to leverage Kafka's group management functionality for automatic consumer load - * balancing and failover. This example assumes that the offsets are stored in Kafka and are manually committed using - * the commit(boolean) API. This example also demonstrates rewinding the consumer's offsets if processing of the consumed - * messages fails. Note that this method of rewinding offsets using {@link #seek(Map) seek(offsets)} is only useful for rewinding the offsets - * of the current consumer instance. As such, this will not trigger a rebalance or affect the fetch offsets for the other consumer instances. + * 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 lets 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 its 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. + * *
- * {@code  
- * Properties props = new Properties();
- * props.put("metadata.broker.list", "localhost:9092");
- * props.put("group.id", "test");
- * props.put("session.timeout.ms", "1000");
- * props.put("enable.auto.commit", "false");
- * KafkaConsumer consumer = new KafkaConsumer(props);
- * consumer.subscribe("foo", "bar");
- * int commitInterval = 100;
- * int numRecords = 0;
- * boolean isRunning = true;
- * Map consumedOffsets = new HashMap();
- * while(isRunning) {
- *     Map records = consumer.poll(100);
- *     try {
- *         Map lastConsumedOffsets = process(records);
- *         consumedOffsets.putAll(lastConsumedOffsets);
- *         numRecords += records.size();
- *         // commit offsets for all partitions of topics foo, bar synchronously, owned by this consumer instance
- *         if(numRecords % commitInterval == 0) 
- *           consumer.commit(false);
- *     } catch(Exception e) {
- *         try {
- *             // rewind consumer's offsets for failed partitions
- *             // assume failedPartitions() returns the list of partitions for which the processing of the last batch of messages failed
- *             List failedPartitions = failedPartitions();   
- *             Map offsetsToRewindTo = new HashMap();
- *             for(TopicPartition failedPartition : failedPartitions) {
- *                 // rewind to the last consumed offset for the failed partition. Since process() failed for this partition, the consumed offset
- *                 // should still be pointing to the last successfully processed offset and hence is the right offset to rewind consumption to.
- *                 offsetsToRewindTo.put(failedPartition, consumedOffsets.get(failedPartition));
+ *     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.commitSync();
+ *                 buffer.clear();
  *             }
- *             // seek to new offsets only for partitions that failed the last process()
- *             consumer.seek(offsetsToRewindTo);
- *         } catch(Exception e) {  break; } // rewind failed
+ *         }
  *     }
- * }         
- * consumer.close();
- * }
  * 
+ * + *

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. *

- * This example demonstrates how to rewind the offsets of the entire consumer group. It is assumed that the user has chosen to use Kafka's - * group management functionality for automatic consumer load balancing and failover. This example also assumes that the offsets are stored in - * Kafka. If group management is used, the right place to systematically rewind offsets for every consumer instance is inside the - * ConsumerRebalanceCallback. The onPartitionsAssigned callback is invoked after the consumer is assigned a new set of partitions on rebalance - * and before the consumption restarts post rebalance. This is the right place to supply the newly rewound offsets to the consumer. It - * is recommended that if you foresee the requirement to ever reset the consumer's offsets in the presence of group management, that you - * always configure the consumer to use the ConsumerRebalanceCallback with a flag that protects whether or not the offset rewind logic is used. - * This method of rewinding offsets is useful if you notice an issue with your message processing after successful consumption and offset commit. - * And you would like to rewind the offsets for the entire consumer group as part of rolling out a fix to your processing logic. In this case, - * you would configure each of your consumer instances with the offset rewind configuration flag turned on and bounce each consumer instance - * in a rolling restart fashion. Each restart will trigger a rebalance and eventually all consumer instances would have rewound the offsets for - * the partitions they own, effectively rewinding the offsets for the entire consumer group. - *

- * {@code  
- * Properties props = new Properties();
- * props.put("metadata.broker.list", "localhost:9092");
- * props.put("group.id", "test");
- * props.put("session.timeout.ms", "1000");
- * props.put("enable.auto.commit", "false");
- * KafkaConsumer consumer = new KafkaConsumer(props,
- *                                            new ConsumerRebalanceCallback() {
- *                                                boolean rewindOffsets = true;  // should be retrieved from external application config
- *                                                public void onPartitionsAssigned(Consumer consumer, Collection partitions) {
- *                                                    Map latestCommittedOffsets = consumer.committed(partitions);
- *                                                    if(rewindOffsets)
- *                                                        Map newOffsets = rewindOffsets(latestCommittedOffsets, 100);
- *                                                    consumer.seek(newOffsets);
- *                                                }
- *                                                public void onPartitionsRevoked(Consumer consumer, Collection partitions) {
- *                                                    consumer.commit(true);
- *                                                }
- *                                                // this API rewinds every partition back by numberOfMessagesToRewindBackTo messages 
- *                                                private Map rewindOffsets(Map currentOffsets,
- *                                                                                                long numberOfMessagesToRewindBackTo) {
- *                                                    Map newOffsets = new HashMap();
- *                                                    for(Map.Entry offset : currentOffsets.entrySet()) 
- *                                                        newOffsets.put(offset.getKey(), offset.getValue() - numberOfMessagesToRewindBackTo);
- *                                                    return newOffsets;
- *                                                }
- *                                            });
- * consumer.subscribe("foo", "bar");
- * int commitInterval = 100;
- * int numRecords = 0;
- * boolean isRunning = true;
- * Map consumedOffsets = new HashMap();
- * while(isRunning) {
- *     Map records = consumer.poll(100);
- *     Map lastConsumedOffsets = process(records);
- *     consumedOffsets.putAll(lastConsumedOffsets);
- *     numRecords += records.size();
- *     // commit offsets for all partitions of topics foo, bar synchronously, owned by this consumer instance
- *     if(numRecords % commitInterval == 0) 
- *         consumer.commit(consumedOffsets, true);
- * }
- * consumer.commit(true);
- * consumer.close();
- * }
- * 
- * This example demonstrates how the consumer can be used to leverage Kafka's group management functionality along with custom offset storage. - * In this example, the assumption made is that the user chooses to store the consumer offsets outside Kafka. This requires the user to - * plugin logic for retrieving the offsets from a custom store and provide the offsets to the consumer in the ConsumerRebalanceCallback - * callback. The onPartitionsAssigned callback is invoked after the consumer is assigned a new set of partitions on rebalance and - * before the consumption restarts post rebalance. This is the right place to supply offsets from a custom store to the consumer. + * 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. + *
*

- * Similarly, the user would also be required to plugin logic for storing the consumer's offsets to a custom store. The onPartitionsRevoked - * callback is invoked right after the consumer has stopped fetching data and before the partition ownership changes. This is the right place - * to commit the offsets for the current set of partitions owned by the consumer. + * This mode is easy to specify, rather than subscribing to the topic, the consumer just subscribes to particular + * partitions: + * *

- * {@code  
- * Properties props = new Properties();
- * props.put("metadata.broker.list", "localhost:9092");
- * props.put("group.id", "test");
- * props.put("session.timeout.ms", "1000");
- * props.put("enable.auto.commit", "false"); // since enable.auto.commit only applies to Kafka based offset storage
- * KafkaConsumer consumer = new KafkaConsumer(props,
- *                                            new ConsumerRebalanceCallback() {
- *                                                public void onPartitionsAssigned(Consumer consumer, Collection partitions) {
- *                                                    Map lastCommittedOffsets = getLastCommittedOffsetsFromCustomStore(partitions);
- *                                                    consumer.seek(lastCommittedOffsets);
- *                                                }
- *                                                public void onPartitionsRevoked(Consumer consumer, Collection partitions) {
- *                                                    Map offsets = getLastConsumedOffsets(partitions);
- *                                                    commitOffsetsToCustomStore(offsets); 
- *                                                }
- *                                                // following APIs should be implemented by the user for custom offset management
- *                                                private Map getLastCommittedOffsetsFromCustomStore(Collection partitions) {
- *                                                    return null;
- *                                                }
- *                                                private Map getLastConsumedOffsets(Collection partitions) { return null; }
- *                                                private void commitOffsetsToCustomStore(Map offsets) {}
- *                                            });
- * consumer.subscribe("foo", "bar");
- * int commitInterval = 100;
- * int numRecords = 0;
- * boolean isRunning = true;
- * Map consumedOffsets = new HashMap();
- * while(isRunning) {
- *     Map records = consumer.poll(100);
- *     Map lastConsumedOffsets = process(records);
- *     consumedOffsets.putAll(lastConsumedOffsets);
- *     numRecords += records.size();
- *     // commit offsets for all partitions of topics foo, bar synchronously, owned by this consumer instance
- *     if(numRecords % commitInterval == 0) 
- *         commitOffsetsToCustomStore(consumedOffsets);
- * }
- * consumer.commit(true);
- * consumer.close();
- * }
+ *     String topic = "foo";
+ *     TopicPartition partition0 = new TopicPartition(topic, 0);
+ *     TopicPartition partition1 = new TopicPartition(topic, 1);
+ *     consumer.assign(partition0);
+ *     consumer.assign(partition1);
  * 
- * This example demonstrates how the consumer can be used to subscribe to specific partitions of certain topics and consume upto the latest - * available message for each of those partitions before shutting down. When used to subscribe to specific partitions, the user foregoes - * the group management functionality and instead relies on manually configuring the consumer instances to subscribe to a set of partitions. - * This example assumes that the user chooses to use Kafka based offset storage. The user still has to specify a group.id to use Kafka - * based offset management. However, session.timeout.ms is not required since the Kafka consumer only does automatic failover when group - * management is used. + * + * 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 its 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 its 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 is + * needed to handle the case where partition assignments change. This can be done by providing a + * {@link ConsumerRebalanceListener} instance in the call to {@link #subscribe(List, ConsumerRebalanceListener)}. + * When partitions are taken from a consumer the consumer will want to commit its offset for those partitions by + * implementing {@link ConsumerRebalanceListener#onPartitionsRevoked(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 ConsumerRebalanceListener#onPartitionsAssigned(Collection)}. + *

+ * Another common use for {@link ConsumerRebalanceListener} 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 its + * position (either automatically or manually). However Kafka allows the consumer to manually control its 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 its 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: + * *

- * {@code  
- * Properties props = new Properties();
- * props.put("metadata.broker.list", "localhost:9092");
- * props.put("group.id", "test");
- * props.put("enable.auto.commit", "true");
- * props.put("auto.commit.interval.ms", "10000");
- * KafkaConsumer consumer = new KafkaConsumer(props);
- * // subscribe to some partitions of topic foo
- * TopicPartition partition0 = new TopicPartition("foo", 0);
- * TopicPartition partition1 = new TopicPartition("foo", 1);
- * TopicPartition[] partitions = new TopicPartition[2];
- * partitions[0] = partition0;
- * partitions[1] = partition1;
- * consumer.subscribe(partitions);
- * // find the last committed offsets for partitions 0,1 of topic foo
- * Map lastCommittedOffsets = consumer.committed(Arrays.asList(partitions));
- * // seek to the last committed offsets to avoid duplicates
- * consumer.seek(lastCommittedOffsets);        
- * // find the offsets of the latest available messages to know where to stop consumption
- * Map latestAvailableOffsets = consumer.offsetsBeforeTime(-2, Arrays.asList(partitions));
- * boolean isRunning = true;
- * Map consumedOffsets = new HashMap();
- * while(isRunning) {
- *     Map records = consumer.poll(100);
- *     Map lastConsumedOffsets = process(records);
- *     consumedOffsets.putAll(lastConsumedOffsets);
- *     for(TopicPartition partition : partitions) {
- *         if(consumedOffsets.get(partition) >= latestAvailableOffsets.get(partition))
- *             isRunning = false;
- *         else
- *             isRunning = true;
+ * 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();
  *     }
- * }
- * consumer.commit(true);
- * consumer.close();
  * }
  * 
- * This example demonstrates how the consumer can be used to subscribe to specific partitions of certain topics and consume upto the latest - * available message for each of those partitions before shutting down. When used to subscribe to specific partitions, the user foregoes - * the group management functionality and instead relies on manually configuring the consumer instances to subscribe to a set of partitions. - * This example assumes that the user chooses to use custom offset storage. + * + * Then in a separate thread, the consumer can be shutdown by setting the closed flag and waking up the consumer. + * *
- * {@code  
- * Properties props = new Properties();
- * props.put("metadata.broker.list", "localhost:9092");
- * KafkaConsumer consumer = new KafkaConsumer(props);
- * // subscribe to some partitions of topic foo
- * TopicPartition partition0 = new TopicPartition("foo", 0);
- * TopicPartition partition1 = new TopicPartition("foo", 1);
- * TopicPartition[] partitions = new TopicPartition[2];
- * partitions[0] = partition0;
- * partitions[1] = partition1;
- * consumer.subscribe(partitions);
- * Map lastCommittedOffsets = getLastCommittedOffsetsFromCustomStore();
- * // seek to the last committed offsets to avoid duplicates
- * consumer.seek(lastCommittedOffsets);        
- * // find the offsets of the latest available messages to know where to stop consumption
- * Map latestAvailableOffsets = consumer.offsetsBeforeTime(-2, Arrays.asList(partitions));
- * boolean isRunning = true;
- * Map consumedOffsets = new HashMap();
- * while(isRunning) {
- *     Map records = consumer.poll(100);
- *     Map lastConsumedOffsets = process(records);
- *     consumedOffsets.putAll(lastConsumedOffsets);
- *     // commit offsets for partitions 0,1 for topic foo to custom store
- *     commitOffsetsToCustomStore(consumedOffsets);
- *     for(TopicPartition partition : partitions) {
- *         if(consumedOffsets.get(partition) >= latestAvailableOffsets.get(partition))
- *             isRunning = false;
- *         else
- *             isRunning = true;
- *     }            
- * }      
- * commitOffsetsToCustomStore(consumedOffsets);   
- * consumer.close();
- * }
+ *     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 its 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 its 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 { +@InterfaceStability.Unstable +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 static final String JMX_PREFIX = "kafka.consumer"; + + private String clientId; + private final Coordinator coordinator; + private final Deserializer keyDeserializer; + private final Deserializer valueDeserializer; + private final Fetcher fetcher; - private final long metadataFetchTimeoutMs; - private final long totalMemorySize; + private final Time time; + private final ConsumerNetworkClient client; private final Metrics metrics; - private final Set subscribedTopics; - private final Set subscribedPartitions; - + private final SubscriptionState subscriptions; + private final Metadata metadata; + private final long retryBackoffMs; + private long requestTimeoutMs; + private boolean closed = false; + private Metadata.Listener metadataListener; + + // 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); + /** * 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 + * 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 + * + * @param configs The consumer configs */ public KafkaConsumer(Map configs) { - this(new ConsumerConfig(configs), null); + this(configs, null, null); } /** - * A consumer is instantiated by providing a set of key-value pairs as configuration and a {@link ConsumerRebalanceCallback} - * implementation + * A consumer is instantiated by providing a set of key-value pairs as configuration, a + * {@link ConsumerRebalanceListener} 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 configs The consumer configs + * @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) { - this(new ConsumerConfig(configs), callback); + public KafkaConsumer(Map configs, + Deserializer keyDeserializer, + Deserializer valueDeserializer) { + this(new ConsumerConfig(ConsumerConfig.addDeserializerToConfig(configs, keyDeserializer, valueDeserializer)), + 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} 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(new ConsumerConfig(properties), null); + this(properties, null, null); } /** - * A consumer is instantiated by providing a {@link java.util.Properties} object as configuration and a - * {@link ConsumerRebalanceCallback} implementation. + * A consumer is instantiated by providing a {@link java.util.Properties} object as configuration and a + * {@link ConsumerRebalanceListener} 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) { - this(new ConsumerConfig(properties), callback); + public KafkaConsumer(Properties properties, + Deserializer keyDeserializer, + Deserializer valueDeserializer) { + this(new ConsumerConfig(ConsumerConfig.addDeserializerToConfig(properties, keyDeserializer, valueDeserializer)), + keyDeserializer, + valueDeserializer); + } + + @SuppressWarnings("unchecked") + private KafkaConsumer(ConsumerConfig config, + Deserializer keyDeserializer, + Deserializer valueDeserializer) { + try { + log.debug("Starting the Kafka consumer"); + this.requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG); + int sessionTimeOutMs = config.getInt(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG); + int fetchMaxWaitMs = config.getInt(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG); + if (this.requestTimeoutMs <= sessionTimeOutMs || this.requestTimeoutMs <= fetchMaxWaitMs) + throw new ConfigException(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG + " should be greater than " + ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG + " and " + ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG); + this.time = new SystemTime(); + + MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ConsumerConfig.METRICS_NUM_SAMPLES_CONFIG)) + .timeWindow(config.getLong(ConsumerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG), + TimeUnit.MILLISECONDS); + clientId = config.getString(ConsumerConfig.CLIENT_ID_CONFIG); + 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(JMX_PREFIX)); + 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); + ChannelBuilder channelBuilder = ClientUtils.createChannelBuilder(config.values()); + NetworkClient netClient = new NetworkClient( + new Selector(config.getLong(ConsumerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, time, metricGrpPrefix, metricsTags, channelBuilder), + 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), + config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_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.getInt(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG), + config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), + this.subscriptions, + metrics, + metricGrpPrefix, + metricsTags, + this.time, + requestTimeoutMs, + retryBackoffMs, + new Coordinator.DefaultOffsetCommitCallback(), + config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG), + config.getLong(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); + if (keyDeserializer == null) { + this.keyDeserializer = config.getConfiguredInstance(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + Deserializer.class); + this.keyDeserializer.configure(config.originals(), true); + } else { + config.ignore(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG); + this.keyDeserializer = keyDeserializer; + } + if (valueDeserializer == null) { + this.valueDeserializer = config.getConfiguredInstance(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + Deserializer.class); + this.valueDeserializer.configure(config.originals(), false); + } else { + config.ignore(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG); + 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(); + AppInfoParser.registerAppInfo(JMX_PREFIX, clientId); + + 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); + } } - private KafkaConsumer(ConsumerConfig config) { - this(config, null); + /** + * The set of partitions currently assigned to this consumer. If subscription happened by directly assigning + * partitions using {@link #assign(List)} then this will simply return the same partitions that + * were assigned. If topic subscription was used, then this will give the set of topic partitions 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). + * @return The set of partitions currently assigned to this consumer + */ + public Set assignment() { + acquire(); + try { + return Collections.unmodifiableSet(new HashSet<>(this.subscriptions.assignedPartitions())); + } finally { + release(); + } } - private KafkaConsumer(ConsumerConfig config, ConsumerRebalanceCallback callback) { - log.trace("Starting the Kafka consumer"); - subscribedTopics = new HashSet(); - subscribedPartitions = new HashSet(); - this.metrics = new Metrics(new MetricConfig(), - Collections.singletonList((MetricsReporter) new JmxReporter("kafka.consumer.")), - new SystemTime()); - this.metadataFetchTimeoutMs = config.getLong(ConsumerConfig.METADATA_FETCH_TIMEOUT_CONFIG); - this.totalMemorySize = config.getLong(ConsumerConfig.TOTAL_BUFFER_MEMORY_CONFIG); - List addresses = ClientUtils.parseAndValidateAddresses(config.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); - config.logUnused(); - log.debug("Kafka consumer started"); + /** + * Get the current subscription. Will return the same topics used in the most recent call to + * {@link #subscribe(List, ConsumerRebalanceListener)}, or an empty set if no such call has been made. + * @return The set of topics currently subscribed to + */ + public Set subscription() { + acquire(); + try { + return Collections.unmodifiableSet(new HashSet<>(this.subscriptions.subscription())); + } finally { + release(); + } } /** - * Incrementally subscribes to the given list of topics and uses the consumer's group management functionality + * Subscribe to the given list of topics and use the consumer's group management functionality to + * assign partitions. Topic subscriptions are not incremental. This list will replace the current + * assignment (if there is one). Note that it is not possible to combine topic subscription with group management + * with manual partition assignment through {@link #assign(List)}. *

- * 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 - + * 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 + *
  • 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 + * + *

    + * When any of these events are triggered, the provided listener will be invoked first to indicate that + * the consumer's assignment has been revoked, and then again when the new assignment has been received. + * Note that this listener will immediately override any listener set in a previous call to subscribe. + * It is guaranteed, however, that the partitions revoked/assigned through this interface are from topics + * subscribed in this call. See {@link ConsumerRebalanceListener} for more details. + * + * @param topics The list of topics to subscribe to + * @param listener Non-null listener instance to get notifications on partition assignment/revocation for the + * subscribed topics */ @Override - public void subscribe(String... topics) { - if(subscribedPartitions.size() > 0) - throw new IllegalStateException("Subcription to topics and partitions is mutually exclusive"); - for(String topic:topics) - subscribedTopics.add(topic); - // TODO: trigger a rebalance operation + public void subscribe(List topics, ConsumerRebalanceListener listener) { + acquire(); + try { + log.debug("Subscribed to topic(s): {}", Utils.join(topics, ", ")); + this.subscriptions.subscribe(topics, listener); + metadata.setTopics(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. + * Subscribe to the given list of topics and use the consumer's group management functionality to + * assign partitions. Topic subscriptions are not incremental. This list will replace the current + * assignment (if there is one). It is not possible to combine topic subscription with group management + * with manual partition assignment through {@link #assign(List)}. *

    - * @param partitions Partitions to incrementally subscribe to + * This is a short-hand for {@link #subscribe(List, ConsumerRebalanceListener)}, which + * uses a noop listener. If you need the ability to either seek to particular offsets, you should prefer + * {@link #subscribe(List, ConsumerRebalanceListener)}, since group rebalances will cause partition offsets + * to be reset. You should also prefer to provide your own listener if you are doing your own offset + * management since the listener gives you an opportunity to commit offsets before a rebalance finishes. + * + * @param topics The list of topics to subscribe to */ @Override - public void subscribe(TopicPartition... partitions) { - if(subscribedTopics.size() > 0) - throw new IllegalStateException("Subcription to topics and partitions is mutually exclusive"); - for(TopicPartition partition:partitions) - subscribedPartitions.add(partition); + public void subscribe(List topics) { + subscribe(topics, new NoOpConsumerRebalanceListener()); } /** - * Unsubscribe from the specific topics. This will trigger a rebalance operation and messages for this topic will not be returned - * from the next {@link #poll(long) poll()} onwards - * @param topics Topics to unsubscribe from + * Subscribes to topics matching specified pattern and uses the consumer's group + * management functionality. The pattern matching will be done periodically against topics + * existing at the time of check. + *

    + * 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 pattern Pattern to subscribe to */ - public void unsubscribe(String... topics) { - // throw an exception if the topic was never subscribed to - for(String topic:topics) { - if(!subscribedTopics.contains(topic)) - throw new IllegalStateException("Topic " + topic + " was never subscribed to. subscribe(" + topic + ") should be called prior" + - " to unsubscribe(" + topic + ")"); - subscribedTopics.remove(topic); + @Override + public void subscribe(Pattern pattern, ConsumerRebalanceListener listener) { + acquire(); + try { + log.debug("Subscribed to pattern: {}", pattern); + metadataListener = new Metadata.Listener() { + @Override + public void onMetadataUpdate(Cluster cluster) { + final List topicsToSubscribe = new ArrayList<>(); + + for (String topic : cluster.topics()) + if (subscriptions.getSubscribedPattern().matcher(topic).matches()) + topicsToSubscribe.add(topic); + + subscriptions.changeSubscription(topicsToSubscribe); + metadata.setTopics(topicsToSubscribe); + } + }; + this.subscriptions.subscribe(pattern, listener); + this.metadata.needMetadataForAllTopics(true); + this.metadata.addListener(metadataListener); + } finally { + release(); } - // TODO trigger a rebalance operation } /** - * Unsubscribe from the specific topic partitions. Messages for these partitions will not be returned from the next - * {@link #poll(long) poll()} onwards - * @param partitions Partitions to unsubscribe from + * Unsubscribe from topics currently subscribed to */ - public void unsubscribe(TopicPartition... partitions) { - // throw an exception if the partition was never subscribed to - for(TopicPartition partition:partitions) { - if(!subscribedPartitions.contains(partition)) - throw new IllegalStateException("Partition " + partition + " was never subscribed to. subscribe(new TopicPartition(" + - partition.topic() + "," + partition.partition() + ") should be called prior" + - " to unsubscribe(new TopicPartition(" + partition.topic() + "," + partition.partition() + ")"); - subscribedPartitions.remove(partition); + public void unsubscribe() { + acquire(); + try { + this.subscriptions.unsubscribe(); + this.coordinator.resetGeneration(); + this.metadata.needMetadataForAllTopics(false); + this.metadata.removeListener(metadataListener); + } finally { + release(); } - // trigger a rebalance operation - } - - /** - * 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(Map) seek(offsets)} - * is used. If {@link #seek(Map) seek(offsets)} 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, boolean) 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, waits indefinitely. Must not be negative + } + + /** + * Assign a list of partition to this consumer. This interface does not allow for incremental assignment + * and will replace the previous assignment (if there is one). + *

    + * Manual topic assignment through this method 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. Note that it is not possible to use both manual partition assignment with {@link #assign(List)} + * and group assignment with {@link #subscribe(List, ConsumerRebalanceListener)}. + * + * @param partitions The list of partitions to assign this consumer + */ + @Override + public void assign(List partitions) { + acquire(); + try { + log.debug("Subscribed to partition(s): {}", Utils.join(partitions, ", ")); + this.subscriptions.assign(partitions); + Set topics = new HashSet<>(); + for (TopicPartition tp : partitions) + topics.add(tp.topic()); + metadata.setTopics(topics); + } finally { + release(); + } + } + + /** + * Fetches data for the topics or partitions specified using one of the subscribe/assign 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 #commitSync(Map) commit(offsets)} 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. + * @throws org.apache.kafka.common.errors.OffsetOutOfRangeException If there is OffsetOutOfRange error in fetchResponse and + * the defaultResetPolicy is NONE */ @Override - public Map poll(long timeout) { - // TODO Auto-generated method stub - return null; + 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); + + 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 -= time.milliseconds() - start; + } + + return ConsumerRecords.empty(); + } finally { + release(); + } } /** - * Commits the specified offsets for the specified list of topics and partitions to Kafka. + * 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) + * @throws org.apache.kafka.common.errors.OffsetOutOfRangeException If there is OffsetOutOfRange error in fetchResponse and + * the defaultResetPolicy is NONE + */ + 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(); + Map>> records = fetcher.fetchedRecords(); + // Avoid block waiting for response if we already have data available, e.g. from another API call to commit. + if (!records.isEmpty()) { + client.poll(0); + return records; + } + fetcher.initFetches(cluster); + client.poll(timeout); + return fetcher.fetchedRecords(); + } + + + + /** + * 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. *

    - * 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. - * @param offsets The list of offsets per partition that should be committed to Kafka. - * @param sync If true, commit will block until the consumer receives an acknowledgment - * @return An {@link OffsetMetadata} object that contains the partition, offset and a corresponding error code. Returns null - * if the sync flag is set to false. + * This is a synchronous commits and will block until either the commit succeeds or an unrecoverable error is + * encountered (in which case it is thrown to the caller). */ @Override - public OffsetMetadata commit(Map offsets, boolean sync) { - throw new UnsupportedOperationException(); + public void commitSync() { + acquire(); + try { + commitSync(subscriptions.allConsumed()); + } finally { + release(); + } } /** - * Commits offsets returned on the last {@link #poll(long) poll()} for the subscribed list of topics and - * partitions. + * 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. *

    - * 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. - * @param sync If true, commit will block until the consumer receives an acknowledgment - * @return An {@link OffsetMetadata} object that contains the partition, offset and a corresponding error code. Returns null - * if the sync flag is set to false. + * This is a synchronous commits and will block until either the commit succeeds or an unrecoverable error is + * encountered (in which case it is thrown to the caller). + * + * @param offsets A map of offsets by partition with associated metadata + */ + @Override + public void commitSync(final Map offsets) { + acquire(); + try { + coordinator.commitOffsetsSync(offsets); + } finally { + release(); + } + } + + /** + * Convenient method. Same as {@link #commitAsync(OffsetCommitCallback) commitAsync(null)} */ @Override - public OffsetMetadata commit(boolean sync) { - throw new UnsupportedOperationException(); + public void commitAsync() { + commitAsync(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 + * 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. + *

    + * This is an asynchronous call and will not block. Any errors encountered are either passed to the callback + * (if provided) or discarded. + * + * @param callback Callback to invoke when the commit completes */ @Override - public void seek(Map offsets) { + public void commitAsync(OffsetCommitCallback callback) { + acquire(); + try { + commitAsync(subscriptions.allConsumed(), callback); + } finally { + release(); + } } /** - * Returns the fetch position of the next message for the specified topic partition to be used on the next {@link #poll(long) poll()} - * @param partitions Partitions for which the fetch position will be returned - * @return The position from which data will be fetched for the specified partition on the next {@link #poll(long) poll()} + * 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. + *

    + * This is an asynchronous call and will not block. Any errors encountered are either passed to the callback + * (if provided) or discarded. + * + * @param offsets A map of offsets by partition with associate metadata. This map will be copied internally, so it + * is safe to mutate the map after returning. + * @param callback Callback to invoke when the commit completes */ - public Map position(Collection partitions) { - return null; + @Override + public void commitAsync(final Map offsets, OffsetCommitCallback callback) { + acquire(); + try { + log.debug("Committing offsets: {} ", offsets); + coordinator.commitOffsetsAsync(new HashMap<>(offsets), callback); + } finally { + release(); + } } /** - * Fetches the last committed offsets of partitions that the consumer currently consumes. This API is only relevant if Kafka based offset - * storage is used. This API can be used in conjunction with {@link #seek(Map) seek(offsets)} to rewind consumption of data. - * @param partitions The list of partitions to return the last committed offset for - * @return The list of offsets committed on the last {@link #commit(boolean) commit(sync)} + * 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 Map committed(Collection partitions) { - // TODO Auto-generated method stub - throw new UnsupportedOperationException(); + 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(); + } } /** - * Fetches offsets before a certain timestamp. Note that the offsets returned are approximately computed and do not correspond to the exact - * message at the given timestamp. As such, if the consumer is rewound to offsets returned by this API, there may be duplicate messages - * returned by the consumer. - * @param partitions The list of partitions for which the offsets are returned - * @param timestamp The unix timestamp. Value -1 indicates earliest available timestamp. Value -2 indicates latest available timestamp. - * @return The offsets per partition before the specified timestamp. + * Seek to the first offset for each of the given partitions */ - public Map offsetsBeforeTime(long timestamp, Collection partitions) { - return null; + 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. This function evaluates lazily, seeking to the + * final offset in all partitions only when poll() or position() are called. + */ + 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.isAssigned(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)); + offset = this.subscriptions.consumed(partition); + } + 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 its cache of committed offsets. + * + * @param partition The partition to check + * @return The last committed offset and metadata or null if there was no prior commit + */ + @Override + public OffsetAndMetadata committed(TopicPartition partition) { + acquire(); + try { + OffsetAndMetadata committed; + if (subscriptions.isAssigned(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); + } + + return committed; + } finally { + release(); + } + } + + /** + * Get the metrics kept by the consumer + */ @Override - public Map metrics() { + 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(); + } + } + + /** + * Get metadata about partitions for all topics. This method will issue a remote call to the + * server. + * + * @return The map of topics and its partitions + */ + @Override + public Map> listTopics() { + acquire(); + try { + return fetcher.getAllTopics(requestTimeoutMs); + } finally { + release(); + } + } + + /** + * Suspend fetching from the requested partitions. Future calls to {@link #poll(long)} will not return + * any records from these partitions until they have been resumed using {@link #resume(TopicPartition...)}. + * Note that this method does not affect partition subscription. In particular, it does not cause a group + * rebalance when automatic assignment is used. + * @param partitions The partitions which should be paused + */ + @Override + public void pause(TopicPartition... partitions) { + acquire(); + try { + for (TopicPartition partition: partitions) { + log.debug("Pausing partition {}", partition); + subscriptions.pause(partition); + } + } finally { + release(); + } + } + + /** + * Resume any partitions which have been paused with {@link #pause(TopicPartition...)}. New calls to + * {@link #poll(long)} will return records from these partitions if there are any to be fetched. + * If the partitions were not previously paused, this method is a no-op. + * @param partitions The partitions which should be resumed + */ + @Override + public void resume(TopicPartition... partitions) { + acquire(); + try { + for (TopicPartition partition: partitions) { + log.debug("Resuming partition {}", partition); + subscriptions.resume(partition); + } + } 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."); - subscribedTopics.clear(); - subscribedPartitions.clear(); - this.metrics.close(); + AtomicReference firstException = new AtomicReference(); + this.closed = true; + ClientUtils.closeQuietly(coordinator, "coordinator", firstException); + 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); + AppInfoParser.unregisterAppInfo(JMX_PREFIX, clientId); log.debug("The Kafka consumer has closed."); + if (firstException.get() != null && !swallowException) { + throw new KafkaException("Failed to close kafka consumer", firstException.get()); + } + } + + /** + * 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 index c3aad3b4d6b67..3c0f26114d029 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java @@ -9,184 +9,369 @@ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES 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.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectOutputStream; +import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +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.errors.TimeoutException; + import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Set; -import org.apache.kafka.common.Metric; -import org.apache.kafka.common.TopicPartition; +import java.util.regex.Pattern; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; /** - * 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. + * A mock of the {@link Consumer} interface you can use for testing code that uses Kafka. This class is not + * threadsafe . However, you can use the {@link #waitForPollThen(Runnable,long)} method to write multithreaded tests + * where a driver thread waits for {@link #poll(long)} to be called and then can safely perform operations during a + * callback. */ -public class MockConsumer implements Consumer { +public class MockConsumer implements Consumer { - private final Set subscribedPartitions; - private final Set subscribedTopics; - private final Map committedOffsets; - private final Map consumedOffsets; - - public MockConsumer() { - subscribedPartitions = new HashSet(); - subscribedTopics = new HashSet(); - committedOffsets = new HashMap(); - consumedOffsets = new HashMap(); + private final Map> partitions; + private final SubscriptionState subscriptions; + private Map>> records; + private Set paused; + private boolean closed; + private final Map beginningOffsets; + private final Map endOffsets; + + private AtomicReference pollLatch; + private KafkaException exception; + + private AtomicBoolean wakeup; + + public MockConsumer(OffsetResetStrategy offsetResetStrategy) { + this.subscriptions = new SubscriptionState(offsetResetStrategy); + this.partitions = new HashMap<>(); + this.records = new HashMap<>(); + this.paused = new HashSet<>(); + this.closed = false; + this.beginningOffsets = new HashMap<>(); + this.endOffsets = new HashMap<>(); + this.pollLatch = new AtomicReference<>(); + this.exception = null; + this.wakeup = new AtomicBoolean(false); } @Override - public void subscribe(String... topics) { - if(subscribedPartitions.size() > 0) - throw new IllegalStateException("Subcription to topics and partitions is mutually exclusive"); - for(String topic : topics) { - subscribedTopics.add(topic); - } + public Set assignment() { + return this.subscriptions.assignedPartitions(); + } + + /** Simulate a rebalance event. */ + public void rebalance(Collection newAssignment) { + // TODO: Rebalance callbacks + this.records.clear(); + this.subscriptions.changePartitionAssignment(newAssignment); } @Override - public void subscribe(TopicPartition... partitions) { - if(subscribedTopics.size() > 0) - throw new IllegalStateException("Subcription to topics and partitions is mutually exclusive"); - for(TopicPartition partition : partitions) { - subscribedPartitions.add(partition); - consumedOffsets.put(partition, 0L); - } + public Set subscription() { + return this.subscriptions.subscription(); } - public void unsubscribe(String... topics) { - // throw an exception if the topic was never subscribed to - for(String topic:topics) { - if(!subscribedTopics.contains(topic)) - throw new IllegalStateException("Topic " + topic + " was never subscribed to. subscribe(" + topic + ") should be called prior" + - " to unsubscribe(" + topic + ")"); - subscribedTopics.remove(topic); - } + @Override + public void subscribe(List topics) { + subscribe(topics, new NoOpConsumerRebalanceListener()); } - public void unsubscribe(TopicPartition... partitions) { - // throw an exception if the partition was never subscribed to - for(TopicPartition partition:partitions) { - if(!subscribedPartitions.contains(partition)) - throw new IllegalStateException("Partition " + partition + " was never subscribed to. subscribe(new TopicPartition(" + - partition.topic() + "," + partition.partition() + ") should be called prior" + - " to unsubscribe(new TopicPartition(" + partition.topic() + "," + partition.partition() + ")"); - subscribedPartitions.remove(partition); - committedOffsets.remove(partition); - consumedOffsets.remove(partition); + @Override + public void subscribe(Pattern pattern, final ConsumerRebalanceListener listener) { + ensureNotClosed(); + this.subscriptions.subscribe(pattern, listener); + List topicsToSubscribe = new ArrayList<>(); + for (String topic: partitions.keySet()) { + if (pattern.matcher(topic).matches() && + !subscriptions.subscription().contains(topic)) + topicsToSubscribe.add(topic); } + ensureNotClosed(); + this.subscriptions.changeSubscription(topicsToSubscribe); } @Override - public Map poll(long timeout) { - // hand out one dummy record, 1 per topic - Map> records = new HashMap>(); - Map recordMetadata = new HashMap(); - for(TopicPartition partition : subscribedPartitions) { - // get the last consumed offset - long messageSequence = consumedOffsets.get(partition); - ByteArrayOutputStream byteStream = new ByteArrayOutputStream(); - ObjectOutputStream outputStream; - try { - outputStream = new ObjectOutputStream(byteStream); - outputStream.writeLong(messageSequence++); - outputStream.close(); - } catch (IOException e) { - e.printStackTrace(); - } - List recordsForTopic = records.get(partition.topic()); - if(recordsForTopic == null) { - recordsForTopic = new ArrayList(); - records.put(partition.topic(), recordsForTopic); + public void subscribe(List topics, final ConsumerRebalanceListener listener) { + ensureNotClosed(); + this.subscriptions.subscribe(topics, listener); + } + + @Override + public void assign(List partitions) { + ensureNotClosed(); + this.subscriptions.assign(partitions); + } + + @Override + public void unsubscribe() { + ensureNotClosed(); + subscriptions.unsubscribe(); + } + + @Override + public ConsumerRecords poll(long timeout) { + ensureNotClosed(); + + CountDownLatch pollLatchCopy = pollLatch.get(); + if (pollLatchCopy != null) { + pollLatch.set(null); + pollLatchCopy.countDown(); + synchronized (pollLatchCopy) { + // Will block until caller of waitUntilPollThen() finishes their callback. } - recordsForTopic.add(new ConsumerRecord(partition.topic(), partition.partition(), null, byteStream.toByteArray(), messageSequence)); - consumedOffsets.put(partition, messageSequence); } - for(Entry> recordsPerTopic : records.entrySet()) { - Map> recordsPerPartition = new HashMap>(); - for(ConsumerRecord record : recordsPerTopic.getValue()) { - List recordsForThisPartition = recordsPerPartition.get(record.partition()); - if(recordsForThisPartition == null) { - recordsForThisPartition = new ArrayList(); - recordsPerPartition.put(record.partition(), recordsForThisPartition); - } - recordsForThisPartition.add(record); + + if (wakeup.get()) { + wakeup.set(false); + throw new ConsumerWakeupException(); + } + + if (exception != null) { + RuntimeException exception = this.exception; + this.exception = null; + throw exception; + } + + // Handle seeks that need to wait for a poll() call to be processed + for (TopicPartition tp : subscriptions.missingFetchPositions()) + updateFetchPosition(tp); + + // update the consumed offset + for (Map.Entry>> entry : this.records.entrySet()) { + if (!subscriptions.isPaused(entry.getKey())) { + List> recs = entry.getValue(); + if (!recs.isEmpty()) + this.subscriptions.consumed(entry.getKey(), recs.get(recs.size() - 1).offset() + 1); } - recordMetadata.put(recordsPerTopic.getKey(), new ConsumerRecords(recordsPerTopic.getKey(), recordsPerPartition)); } - return recordMetadata; + + ConsumerRecords copy = new ConsumerRecords(this.records); + this.records = new HashMap>>(); + return copy; + } + + public void addRecord(ConsumerRecord record) { + ensureNotClosed(); + TopicPartition tp = new TopicPartition(record.topic(), record.partition()); + Set currentAssigned = new HashSet<>(this.subscriptions.assignedPartitions()); + if (!currentAssigned.contains(tp)) + throw new IllegalStateException("Cannot add records for a partition that is not assigned to the consumer"); + List> recs = this.records.get(tp); + if (recs == null) { + recs = new ArrayList>(); + this.records.put(tp, recs); + } + recs.add(record); + } + + public void setException(KafkaException exception) { + this.exception = exception; } @Override - public OffsetMetadata commit(Map offsets, boolean sync) { - if(!sync) - return null; - for(Entry partitionOffset : offsets.entrySet()) { - committedOffsets.put(partitionOffset.getKey(), partitionOffset.getValue()); - } - return new OffsetMetadata(committedOffsets, null); + public void commitAsync(Map offsets, OffsetCommitCallback callback) { + ensureNotClosed(); + for (Map.Entry entry : offsets.entrySet()) + subscriptions.committed(entry.getKey(), entry.getValue()); + if (callback != null) { + callback.onComplete(offsets, null); + } } @Override - public OffsetMetadata commit(boolean sync) { - if(!sync) - return null; - return commit(consumedOffsets, sync); + public void commitSync(Map offsets) { + commitAsync(offsets, null); } @Override - public void seek(Map offsets) { - // change the fetch offsets - for(Entry partitionOffset : offsets.entrySet()) { - consumedOffsets.put(partitionOffset.getKey(), partitionOffset.getValue()); - } + public void commitAsync() { + commitAsync(null); } @Override - public Map committed(Collection partitions) { - Map offsets = new HashMap(); - for(TopicPartition partition : partitions) { - offsets.put(new TopicPartition(partition.topic(), partition.partition()), committedOffsets.get(partition)); - } - return offsets; + public void commitAsync(OffsetCommitCallback callback) { + ensureNotClosed(); + commitAsync(this.subscriptions.allConsumed(), callback); + } + + @Override + public void commitSync() { + commitSync(this.subscriptions.allConsumed()); + } + + @Override + public void seek(TopicPartition partition, long offset) { + ensureNotClosed(); + subscriptions.seek(partition, offset); + } + + @Override + public OffsetAndMetadata committed(TopicPartition partition) { + ensureNotClosed(); + return subscriptions.committed(partition); } @Override - public Map position(Collection partitions) { - Map positions = new HashMap(); - for(TopicPartition partition : partitions) { - positions.put(partition, consumedOffsets.get(partition)); + public long position(TopicPartition partition) { + ensureNotClosed(); + if (!this.subscriptions.isAssigned(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) { + updateFetchPosition(partition); + offset = this.subscriptions.consumed(partition); } - return positions; + return offset; } @Override - public Map offsetsBeforeTime(long timestamp, - Collection partitions) { - throw new UnsupportedOperationException(); + public void seekToBeginning(TopicPartition... partitions) { + ensureNotClosed(); + for (TopicPartition tp : partitions) + subscriptions.needOffsetReset(tp, OffsetResetStrategy.EARLIEST); + } + + public void updateBeginningOffsets(Map newOffsets) { + beginningOffsets.putAll(newOffsets); } @Override - public Map metrics() { - return null; + public void seekToEnd(TopicPartition... partitions) { + ensureNotClosed(); + for (TopicPartition tp : partitions) + subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST); + } + + public void updateEndOffsets(Map newOffsets) { + endOffsets.putAll(newOffsets); + } + + @Override + public Map metrics() { + ensureNotClosed(); + return Collections.emptyMap(); + } + + @Override + public List partitionsFor(String topic) { + ensureNotClosed(); + List parts = this.partitions.get(topic); + if (parts == null) + return Collections.emptyList(); + else + return parts; + } + + @Override + public Map> listTopics() { + ensureNotClosed(); + return partitions; + } + + public void updatePartitions(String topic, List partitions) { + ensureNotClosed(); + this.partitions.put(topic, partitions); + } + + @Override + public void pause(TopicPartition... partitions) { + for (TopicPartition partition : partitions) { + subscriptions.pause(partition); + paused.add(partition); + } + } + + @Override + public void resume(TopicPartition... partitions) { + for (TopicPartition partition : partitions) { + subscriptions.resume(partition); + paused.remove(partition); + } } @Override public void close() { - // unsubscribe from all partitions - TopicPartition[] allPartitions = new TopicPartition[subscribedPartitions.size()]; - unsubscribe(subscribedPartitions.toArray(allPartitions)); + ensureNotClosed(); + this.closed = true; + } + + public boolean closed() { + return this.closed; + } + + @Override + public void wakeup() { + wakeup.set(true); + } + + public void waitForPoll(long timeoutMs) { + waitForPollThen(null, timeoutMs); + } + + public void waitForPollThen(Runnable task, long timeoutMs) { + CountDownLatch latch = new CountDownLatch(1); + synchronized (latch) { + pollLatch.set(latch); + try { + if (!latch.await(timeoutMs, TimeUnit.MILLISECONDS)) + throw new TimeoutException("Timed out waiting for consumer thread to call poll()."); + } catch (InterruptedException e) { + throw new IllegalStateException("MockConsumer waiting thread was interrupted.", e); + } + if (task != null) + task.run(); + } + } + + public Set paused() { + return Collections.unmodifiableSet(new HashSet<>(paused)); + } + + private void ensureNotClosed() { + if (this.closed) + throw new IllegalStateException("This consumer has already been closed."); + } + + private void updateFetchPosition(TopicPartition tp) { + if (subscriptions.isOffsetResetNeeded(tp)) { + resetOffsetPosition(tp); + } else if (subscriptions.committed(tp) == null) { + subscriptions.needOffsetReset(tp); + resetOffsetPosition(tp); + } else { + subscriptions.seek(tp, subscriptions.committed(tp).offset()); + } + } + + private void resetOffsetPosition(TopicPartition tp) { + OffsetResetStrategy strategy = subscriptions.resetStrategy(tp); + Long offset; + if (strategy == OffsetResetStrategy.EARLIEST) { + offset = beginningOffsets.get(tp); + if (offset == null) + throw new IllegalStateException("MockConsumer didn't have beginning offset specified, but tried to seek to beginning"); + } else if (strategy == OffsetResetStrategy.LATEST) { + offset = endOffsets.get(tp); + if (offset == null) + throw new IllegalStateException("MockConsumer didn't have end offset specified, but tried to seek to end"); + } else { + throw new NoOffsetForPartitionException("No offset available"); + } + seek(tp, offset); } } 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/OffsetAndMetadata.java b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndMetadata.java new file mode 100644 index 0000000000000..1a9304798bd2f --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndMetadata.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.clients.consumer; + +/** + * The Kafka offset commit API allows users to provide additional metadata (in the form of a string) + * when an offset is committed. This can be useful (for example) to store information about which + * node made the commit, what time the commit was made, etc. + */ +public class OffsetAndMetadata { + private final long offset; + private final String metadata; + + /** + * Construct a new OffsetAndMetadata object for committing through {@link KafkaConsumer}. + * @param offset The offset to be committed + * @param metadata Non-null metadata + */ + public OffsetAndMetadata(long offset, String metadata) { + if (metadata == null) + throw new IllegalArgumentException("Metadata cannot be null"); + + this.offset = offset; + this.metadata = metadata; + } + + /** + * Construct a new OffsetAndMetadata object for committing through {@link KafkaConsumer}. The metadata + * associated with the commit will be empty. + * @param offset The offset to be committed + */ + public OffsetAndMetadata(long offset) { + this(offset, ""); + } + + public long offset() { + return offset; + } + + public String metadata() { + return metadata; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + OffsetAndMetadata that = (OffsetAndMetadata) o; + + if (offset != that.offset) return false; + return metadata == null ? that.metadata == null : metadata.equals(that.metadata); + + } + + @Override + public int hashCode() { + int result = (int) (offset ^ (offset >>> 32)); + result = 31 * result + (metadata != null ? metadata.hashCode() : 0); + return result; + } + + @Override + public String toString() { + return "OffsetAndMetadata{" + + "offset=" + offset + + ", metadata='" + metadata + '\'' + + '}'; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetCommitCallback.java b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetCommitCallback.java new file mode 100644 index 0000000000000..97a06ad48015f --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetCommitCallback.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 OffsetCommitCallback { + + /** + * 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 and associated metadata 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/OffsetMetadata.java b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetMetadata.java deleted file mode 100644 index ea423ad15eebd..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetMetadata.java +++ /dev/null @@ -1,59 +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.consumer; - -import java.util.Map; - -import org.apache.kafka.common.TopicPartition; - -/** - * The metadata for an offset commit that has been acknowledged by the server - */ -public final class OffsetMetadata { - - private final Map offsets; - private final Map errors; - - public OffsetMetadata(Map offsets, Map errors) { - super(); - this.offsets = offsets; - this.errors = errors; - } - - public OffsetMetadata(Map offsets) { - this(offsets, null); - } - - /** - * The offset of the record in the topic/partition. - */ - public long offset(TopicPartition partition) { - if(this.errors != null) - throw errors.get(partition); - return offsets.get(partition); - } - - /** - * @return The exception corresponding to the error code returned by the server - */ - public Exception error(TopicPartition partition) { - if(errors != null) - return errors.get(partition); - else - return null; - } -} 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..0b611fb1ea216 --- /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, now); + 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..701a81bf586c0 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java @@ -0,0 +1,836 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES 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.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.OffsetCommitCallback; +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.errors.UnknownConsumerIdException; +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.io.Closeable; +import java.util.ArrayList; +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 implements Closeable { + + 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 OffsetCommitCallback defaultOffsetCommitCallback; + private final boolean autoCommitEnabled; + + private Node consumerCoordinator; + private String consumerId; + private int generation; + + /** + * Initialize the coordination manager. + */ + public Coordinator(ConsumerNetworkClient client, + String groupId, + int sessionTimeoutMs, + int heartbeatIntervalMs, + String assignmentStrategy, + SubscriptionState subscriptions, + Metrics metrics, + String metricGrpPrefix, + Map metricTags, + Time time, + long requestTimeoutMs, + long retryBackoffMs, + OffsetCommitCallback defaultOffsetCommitCallback, + boolean autoCommitEnabled, + long autoCommitIntervalMs) { + this.client = client; + this.time = time; + this.generation = OffsetCommitRequest.DEFAULT_GENERATION_ID; + 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, heartbeatIntervalMs, time.milliseconds()); + this.heartbeatTask = new HeartbeatTask(); + this.sensors = new CoordinatorMetrics(metrics, metricGrpPrefix, metricTags); + this.requestTimeoutMs = requestTimeoutMs; + this.retryBackoffMs = retryBackoffMs; + this.defaultOffsetCommitCallback = defaultOffsetCommitCallback; + this.autoCommitEnabled = autoCommitEnabled; + + if (autoCommitEnabled) + scheduleAutoCommitTask(autoCommitIntervalMs); + } + + /** + * 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(); + // verify assignment is still active + if (subscriptions.isAssigned(tp)) + 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(); + + // 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; + + // commit offsets prior to rebalance if auto-commit enabled + maybeAutoCommitOffsetsSync(); + + ConsumerRebalanceListener listener = subscriptions.listener(); + + // execute the user's listener before rebalance + log.debug("Revoking previously assigned partitions {}", this.subscriptions.assignedPartitions()); + try { + Set revoked = new HashSet<>(subscriptions.assignedPartitions()); + listener.onPartitionsRevoked(revoked); + } catch (Exception e) { + log.error("User provided listener " + listener.getClass().getName() + + " failed on partition revocation: ", e); + } + + reassignPartitions(); + + // execute the user's listener after rebalance + log.debug("Setting newly assigned partitions {}", this.subscriptions.assignedPartitions()); + try { + Set assigned = new HashSet<>(subscriptions.assignedPartitions()); + listener.onPartitionsAssigned(assigned); + } catch (Exception e) { + log.error("User provided listener " + listener.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.exception() instanceof UnknownConsumerIdException) + continue; + else 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(); + } + } + + + @Override + public void close() { + // commit offsets prior to closing if auto-commit enabled + maybeAutoCommitOffsetsSync(); + } + + 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.subscription()); + 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())); + } + } + } + + public void commitOffsetsAsync(final Map offsets, OffsetCommitCallback callback) { + this.subscriptions.needRefreshCommits(); + RequestFuture future = sendOffsetCommitRequest(offsets); + final OffsetCommitCallback cb = callback == null ? defaultOffsetCommitCallback : callback; + future.addListener(new RequestFutureListener() { + @Override + public void onSuccess(Void value) { + cb.onComplete(offsets, null); + } + + @Override + public void onFailure(RuntimeException e) { + cb.onComplete(offsets, e); + } + }); + } + + public void commitOffsetsSync(Map offsets) { + if (offsets.isEmpty()) + return; + + while (true) { + ensureCoordinatorKnown(); + + RequestFuture future = sendOffsetCommitRequest(offsets); + client.poll(future); + + if (future.succeeded()) { + return; + } + + if (!future.isRetriable()) { + throw future.exception(); + } + + Utils.sleep(retryBackoffMs); + } + } + + private void scheduleAutoCommitTask(final long interval) { + DelayedTask task = new DelayedTask() { + public void run(long now) { + commitOffsetsAsync(subscriptions.allConsumed(), new OffsetCommitCallback() { + @Override + public void onComplete(Map offsets, Exception exception) { + if (exception != null) + log.error("Auto offset commit failed.", exception); + } + }); + client.schedule(this, now + interval); + } + }; + client.schedule(task, time.milliseconds() + interval); + } + + private void maybeAutoCommitOffsetsSync() { + if (autoCommitEnabled) { + try { + commitOffsetsSync(subscriptions.allConsumed()); + } catch (Exception e) { + // consistent with async auto-commit failures, we do not propagate the exception + log.error("Auto offset commit failed.", e); + } + } + } + + /** + * Reset the generation/consumerId tracked by this consumer. + */ + public void resetGeneration() { + this.generation = OffsetCommitRequest.DEFAULT_GENERATION_ID; + this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; + } + + /** + * 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 = new HashMap<>(offsets.size()); + for (Map.Entry entry : offsets.entrySet()) { + OffsetAndMetadata offsetAndMetadata = entry.getValue(); + offsetData.put(entry.getKey(), new OffsetCommitRequest.PartitionData( + offsetAndMetadata.offset(), offsetAndMetadata.metadata())); + } + + 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)); + } + + public static class DefaultOffsetCommitCallback implements OffsetCommitCallback { + @Override + public void onComplete(Map offsets, Exception exception) { + if (exception != null) + log.error("Offset commit failed.", exception); + } + } + + 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(); + OffsetAndMetadata offsetAndMetadata = this.offsets.get(tp); + long offset = offsetAndMetadata.offset(); + + short errorCode = entry.getValue(); + if (errorCode == Errors.NONE.code()) { + log.debug("Committed offset {} for partition {}", offset, tp); + if (subscriptions.isAssigned(tp)) + // update the local cache only if the partition is still assigned + subscriptions.committed(tp, offsetAndMetadata); + } else { + if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() + || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { + coordinatorDead(); + } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code() + || errorCode == Errors.ILLEGAL_GENERATION.code()) { + // need to re-join group + subscriptions.needReassignment(); + } + + log.error("Error committing partition {} at offset {}: {}", + tp, + offset, + Errors.forCode(errorCode).exception().getMessage()); + + future.raise(Errors.forCode(errorCode)); + return; + } + } + + 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, new OffsetAndMetadata(data.offset, data.metadata)); + } 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.REBALANCE_IN_PROGRESS.code()) { + log.info("Attempt to heart beat failed since the group is rebalancing, try to re-join group."); + subscriptions.needReassignment(); + future.raise(Errors.REBALANCE_IN_PROGRESS); + } 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); + } + } + + + 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..4608959f01434 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -0,0 +1,621 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES 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.errors.OffsetOutOfRangeException; +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.requests.MetadataRequest; +import org.apache.kafka.common.requests.MetadataResponse; +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 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; + + private final Map offsetOutOfRangePartitions; + + 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.offsetOutOfRangePartitions = new HashMap<>(); + + 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 + * @throws NoOffsetForPartitionException If no offset is stored for a given partition and no reset policy is available + */ + public void updateFetchPositions(Set partitions) { + // reset the fetch position to the committed position + for (TopicPartition tp : partitions) { + if (!subscriptions.isAssigned(tp) || subscriptions.isFetchable(tp)) + 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 { + long committed = subscriptions.committed(tp).offset(); + log.debug("Resetting offset for partition {} to the committed offset {}", tp, committed); + subscriptions.seek(tp, committed); + } + } + } + + + + /** + * Get metadata for all topics present in Kafka cluster + * + * @param timeout time for which getting all topics is attempted + * @return The map of topics and its partitions + */ + public Map> getAllTopics(long timeout) { + final HashMap> topicsPartitionInfos = new HashMap<>(); + long startTime = time.milliseconds(); + + while (time.milliseconds() - startTime < timeout) { + RequestFuture requestFuture = sendMetadataRequest(); + if (requestFuture != null) { + client.poll(requestFuture); + + if (requestFuture.succeeded()) { + MetadataResponse response = + new MetadataResponse(requestFuture.value().responseBody()); + + for (String topic : response.cluster().topics()) + topicsPartitionInfos.put( + topic, response.cluster().availablePartitionsForTopic(topic)); + + return topicsPartitionInfos; + } + + if (!requestFuture.isRetriable()) + throw requestFuture.exception(); + } + + Utils.sleep(retryBackoffMs); + } + + return topicsPartitionInfos; + } + + /** + * Send Metadata Request to least loaded node in Kafka cluster asynchronously + * @return A future that indicates result of sent metadata request + */ + public RequestFuture sendMetadataRequest() { + final Node node = client.leastLoadedNode(); + return node == null ? null : + client.send( + node, ApiKeys.METADATA, new MetadataRequest(Collections.emptyList())); + } + + /** + * 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 = ListOffsetRequest.EARLIEST_TIMESTAMP; + else if (strategy == OffsetResetStrategy.LATEST) + timestamp = ListOffsetRequest.LATEST_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); + + // we might lose the assignment while fetching the offset, so check it is still active + if (subscriptions.isAssigned(partition)) + 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); + } + } + + /** + * If any partition from previous fetchResponse contains OffsetOutOfRange error and + * the defaultResetPolicy is NONE, throw OffsetOutOfRangeException + * + * @throws OffsetOutOfRangeException If there is OffsetOutOfRange error in fetchResponse + */ + private void throwIfOffsetOutOfRange() throws OffsetOutOfRangeException { + Map currentOutOfRangePartitions = new HashMap<>(); + + // filter offsetOutOfRangePartitions to retain only the fetchable partitions + for (Map.Entry entry: this.offsetOutOfRangePartitions.entrySet()) { + if (!subscriptions.isFetchable(entry.getKey())) { + log.debug("Ignoring fetched records for {} since it is no longer fetchable", entry.getKey()); + continue; + } + Long consumed = subscriptions.consumed(entry.getKey()); + // ignore partition if its consumed offset != offset in fetchResponse, e.g. after seek() + if (consumed != null && entry.getValue().equals(consumed)) + currentOutOfRangePartitions.put(entry.getKey(), entry.getValue()); + } + this.offsetOutOfRangePartitions.clear(); + if (!currentOutOfRangePartitions.isEmpty()) + throw new OffsetOutOfRangeException(currentOutOfRangePartitions); + } + + /** + * Return the fetched records, empty the record buffer and update the consumed position. + * + * @return The fetched records per partition + * @throws OffsetOutOfRangeException If there is OffsetOutOfRange error in fetchResponse and + * the defaultResetPolicy is NONE + */ + public Map>> fetchedRecords() { + if (this.subscriptions.partitionAssignmentNeeded()) { + return Collections.emptyMap(); + } else { + Map>> drained = new HashMap<>(); + throwIfOffsetOutOfRange(); + + for (PartitionRecords part : this.records) { + if (!subscriptions.isFetchable(part.partition)) { + log.debug("Ignoring fetched records for {} since it is no longer fetchable", part.partition); + continue; + } + + Long consumed = subscriptions.consumed(part.partition); + if (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.fetchablePartitions()) { + 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 fetched = this.subscriptions.fetched(partition); + long consumed = this.subscriptions.consumed(partition); + // Only fetch data for partitions whose previously fetched data has been consumed + if (consumed == fetched) + fetch.put(partition, new FetchRequest.PartitionData(fetched, 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()) { + long fetchOffset = request.fetchData().get(tp).offset; + + // we are interested in this fetch only if the beginning offset matches the + // current consumed position + Long consumed = subscriptions.consumed(tp); + if (consumed == null) { + continue; + } else if (consumed != fetchOffset) { + // the fetched position has gotten out of sync with the consumed position + // (which might happen when a rebalance occurs with a fetch in-flight), + // so we need to reset the fetch position so the next fetch is right + subscriptions.fetched(tp, consumed); + continue; + } + + int bytes = 0; + ByteBuffer buffer = partition.recordSet; + MemoryRecords records = MemoryRecords.readableRecords(buffer); + List> parsed = new ArrayList>(); + for (LogEntry logEntry : records) { + parsed.add(parseRecord(tp, logEntry)); + bytes += logEntry.size(); + } + if (!parsed.isEmpty()) { + 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()) { + long fetchOffset = request.fetchData().get(tp).offset; + if (subscriptions.hasDefaultOffsetResetPolicy()) + subscriptions.needOffsetReset(tp); + else + this.offsetOutOfRangePartitions.put(tp, fetchOffset); + log.info("Fetch offset {} is out of range, resetting offset", subscriptions.fetched(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.fetchThrottleTimeSensor.record(response.getThrottleTime()); + } + 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 final Sensor fetchThrottleTimeSensor; + + + 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()); + + this.fetchThrottleTimeSensor = metrics.sensor("fetch-throttle-time"); + this.fetchThrottleTimeSensor.add(new MetricName("fetch-throttle-time-avg", + this.metricGrpName, + "The average throttle time in ms", + tags), new Avg()); + + this.fetchThrottleTimeSensor.add(new MetricName("fetch-throttle-time-max", + this.metricGrpName, + "The maximum throttle time in ms", + 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..79e17e219f09f --- /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 { + private final long timeout; + private final long interval; + + private long lastHeartbeatSend; + private long lastHeartbeatReceive; + private long lastSessionReset; + + public Heartbeat(long timeout, + long interval, + long now) { + if (interval >= timeout) + throw new IllegalArgumentException("Heartbeat must be set lower than the session timeout"); + + this.timeout = timeout; + this.interval = interval; + 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); + + if (timeSinceLastHeartbeat > interval) + return 0; + else + return interval - timeSinceLastHeartbeat; + } + + public boolean sessionTimeoutExpired(long now) { + return now - Math.max(lastSessionReset, lastHeartbeatReceive) > timeout; + } + + public long interval() { + return 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/NoOpConsumerRebalanceListener.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceListener.java new file mode 100644 index 0000000000000..3cb152de92dc4 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceListener.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.internals; + +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.common.TopicPartition; + +import java.util.Collection; + +public class NoOpConsumerRebalanceListener implements ConsumerRebalanceListener { + + @Override + public void onPartitionsAssigned(Collection partitions) {} + + @Override + public void onPartitionsRevoked(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..f5c1afcba49db --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java @@ -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 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) { + if (isDone) + throw new IllegalStateException("Invalid attempt to complete a request future which is already complete"); + 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) { + if (isDone) + throw new IllegalStateException("Invalid attempt to complete a request future which is already complete"); + 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..25a0e9071f5d6 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java @@ -0,0 +1,383 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES 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.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.TopicPartition; + +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.regex.Pattern; + +/** + * A class for tracking the topics, partitions, and offsets for the consumer. A partition + * is "assigned" either directly with {@link #assign(List)} (manual assignment) + * or with {@link #changePartitionAssignment(List)} (automatic assignment). + * + * Once assigned, the partition is not considered "fetchable" until its initial position has + * been set with {@link #seek(TopicPartition, long)}. Fetchable partitions track a fetch + * position which is used to set the offset of the next fetch, and a consumed position + * which is the last offset that has been returned to the user. You can suspend fetching + * from a partition through {@link #pause(TopicPartition)} without affecting the fetched/consumed + * offsets. The partition will remain unfetchable until the {@link #resume(TopicPartition)} is + * used. You can also query the pause state independently with {@link #isPaused(TopicPartition)}. + * + * Note that pause state as well as fetch/consumed positions are not preserved when partition + * assignment is changed whether directly by the user or through a group rebalance. + * + * This class also maintains a cache of the latest commit position for each of the assigned + * partitions. This is updated through {@link #committed(TopicPartition, OffsetAndMetadata)} and can be used + * to set the initial fetch position (e.g. {@link Fetcher#resetOffset(TopicPartition)}. + */ +public class SubscriptionState { + + /* the pattern user has requested */ + private Pattern subscribedPattern; + + /* the list of topics the user has requested */ + private final Set subscription; + + /* the list of partitions the user has requested */ + private final Set userAssignment; + + /* the list of partitions currently assigned */ + private final Map assignment; + + /* 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; + + /* Default offset reset strategy */ + private final OffsetResetStrategy defaultResetStrategy; + + /* Listener to be invoked when assignment changes */ + private ConsumerRebalanceListener listener; + + private static final String SUBSCRIPTION_EXCEPTION_MESSAGE = + "Subscription to topics, partitions and pattern are mutually exclusive"; + + public SubscriptionState(OffsetResetStrategy defaultResetStrategy) { + this.defaultResetStrategy = defaultResetStrategy; + this.subscription = new HashSet<>(); + this.userAssignment = new HashSet<>(); + this.assignment = new HashMap<>(); + this.needsPartitionAssignment = false; + this.needsFetchCommittedOffsets = true; // initialize to true for the consumers to fetch offset upon starting up + this.subscribedPattern = null; + } + + public void subscribe(List topics, ConsumerRebalanceListener listener) { + if (listener == null) + throw new IllegalArgumentException("RebalanceListener cannot be null"); + + if (!this.userAssignment.isEmpty() || this.subscribedPattern != null) + throw new IllegalStateException(SUBSCRIPTION_EXCEPTION_MESSAGE); + + this.listener = listener; + + changeSubscription(topics); + } + + public void changeSubscription(List topicsToSubscribe) { + if (!this.subscription.equals(new HashSet<>(topicsToSubscribe))) { + this.subscription.clear(); + this.subscription.addAll(topicsToSubscribe); + this.needsPartitionAssignment = true; + + // Remove any assigned partitions which are no longer subscribed to + for (Iterator it = assignment.keySet().iterator(); it.hasNext(); ) { + TopicPartition tp = it.next(); + if (!subscription.contains(tp.topic())) + it.remove(); + } + } + + } + + public void needReassignment() { + this.needsPartitionAssignment = true; + } + + public void assign(List partitions) { + if (!this.subscription.isEmpty() || this.subscribedPattern != null) + throw new IllegalStateException(SUBSCRIPTION_EXCEPTION_MESSAGE); + + this.userAssignment.clear(); + this.userAssignment.addAll(partitions); + + for (TopicPartition partition : partitions) + if (!assignment.containsKey(partition)) + addAssignedPartition(partition); + + this.assignment.keySet().retainAll(this.userAssignment); + } + + public void subscribe(Pattern pattern, ConsumerRebalanceListener listener) { + if (listener == null) + throw new IllegalArgumentException("RebalanceListener cannot be null"); + + if (!this.subscription.isEmpty() || !this.userAssignment.isEmpty()) + throw new IllegalStateException(SUBSCRIPTION_EXCEPTION_MESSAGE); + + this.listener = listener; + this.subscribedPattern = pattern; + } + + public void unsubscribe() { + this.subscription.clear(); + this.assignment.clear(); + this.needsPartitionAssignment = true; + this.subscribedPattern = null; + } + + + public Pattern getSubscribedPattern() { + return this.subscribedPattern; + } + + public void clearAssignment() { + this.assignment.clear(); + this.needsPartitionAssignment = !subscription().isEmpty(); + } + + public Set subscription() { + return this.subscription; + } + + public Long fetched(TopicPartition tp) { + return assignedState(tp).fetched; + } + + public void fetched(TopicPartition tp, long offset) { + assignedState(tp).fetched(offset); + } + + private TopicPartitionState assignedState(TopicPartition tp) { + TopicPartitionState state = this.assignment.get(tp); + if (state == null) + throw new IllegalStateException("No current assignment for partition " + tp); + return state; + } + + public void committed(TopicPartition tp, OffsetAndMetadata offset) { + assignedState(tp).committed(offset); + } + + public OffsetAndMetadata committed(TopicPartition tp) { + return assignedState(tp).committed; + } + + 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) { + assignedState(tp).seek(offset); + } + + public Set assignedPartitions() { + return this.assignment.keySet(); + } + + public Set fetchablePartitions() { + Set fetchable = new HashSet<>(); + for (Map.Entry entry : assignment.entrySet()) { + if (entry.getValue().isFetchable()) + fetchable.add(entry.getKey()); + } + return fetchable; + } + + public boolean partitionsAutoAssigned() { + return !this.subscription.isEmpty(); + } + + public void consumed(TopicPartition tp, long offset) { + assignedState(tp).consumed(offset); + } + + public Long consumed(TopicPartition tp) { + return assignedState(tp).consumed; + } + + public Map allConsumed() { + Map allConsumed = new HashMap<>(); + for (Map.Entry entry : assignment.entrySet()) { + TopicPartitionState state = entry.getValue(); + if (state.hasValidPosition) + allConsumed.put(entry.getKey(), new OffsetAndMetadata(state.consumed)); + } + return allConsumed; + } + + public void needOffsetReset(TopicPartition partition, OffsetResetStrategy offsetResetStrategy) { + assignedState(partition).awaitReset(offsetResetStrategy); + } + + public void needOffsetReset(TopicPartition partition) { + needOffsetReset(partition, defaultResetStrategy); + } + + public boolean hasDefaultOffsetResetPolicy() { + return defaultResetStrategy != OffsetResetStrategy.NONE; + } + + public boolean isOffsetResetNeeded(TopicPartition partition) { + return assignedState(partition).awaitingReset; + } + + public OffsetResetStrategy resetStrategy(TopicPartition partition) { + return assignedState(partition).resetStrategy; + } + + public boolean hasAllFetchPositions() { + for (TopicPartitionState state : assignment.values()) + if (!state.hasValidPosition) + return false; + return true; + } + + public Set missingFetchPositions() { + Set missing = new HashSet<>(); + for (Map.Entry entry : assignment.entrySet()) + if (!entry.getValue().hasValidPosition) + missing.add(entry.getKey()); + return missing; + } + + public boolean partitionAssignmentNeeded() { + return this.needsPartitionAssignment; + } + + public void changePartitionAssignment(Collection assignments) { + for (TopicPartition tp : assignments) + if (!this.subscription.contains(tp.topic())) + throw new IllegalArgumentException("Assigned partition " + tp + " for non-subscribed topic."); + this.clearAssignment(); + for (TopicPartition tp: assignments) + addAssignedPartition(tp); + this.needsPartitionAssignment = false; + } + + public boolean isAssigned(TopicPartition tp) { + return assignment.containsKey(tp); + } + + public boolean isPaused(TopicPartition tp) { + return isAssigned(tp) && assignedState(tp).paused; + } + + public boolean isFetchable(TopicPartition tp) { + return isAssigned(tp) && assignedState(tp).isFetchable(); + } + + public void pause(TopicPartition tp) { + assignedState(tp).pause(); + } + + public void resume(TopicPartition tp) { + assignedState(tp).resume(); + } + + private void addAssignedPartition(TopicPartition tp) { + this.assignment.put(tp, new TopicPartitionState()); + } + + public ConsumerRebalanceListener listener() { + return listener; + } + + private static class TopicPartitionState { + private Long consumed; // offset exposed to the user + private Long fetched; // current fetch position + private OffsetAndMetadata committed; // last committed position + + private boolean hasValidPosition; // whether we have valid consumed and fetched positions + private boolean paused; // whether this partition has been paused by the user + private boolean awaitingReset; // whether we are awaiting reset + private OffsetResetStrategy resetStrategy; // the reset strategy if awaitingReset is set + + public TopicPartitionState() { + this.paused = false; + this.consumed = null; + this.fetched = null; + this.committed = null; + this.awaitingReset = false; + this.hasValidPosition = false; + this.resetStrategy = null; + } + + private void awaitReset(OffsetResetStrategy strategy) { + this.awaitingReset = true; + this.resetStrategy = strategy; + this.consumed = null; + this.fetched = null; + this.hasValidPosition = false; + } + + private void seek(long offset) { + this.consumed = offset; + this.fetched = offset; + this.awaitingReset = false; + this.resetStrategy = null; + this.hasValidPosition = true; + } + + private void fetched(long offset) { + if (!hasValidPosition) + throw new IllegalStateException("Cannot update fetch position without valid consumed/fetched positions"); + this.fetched = offset; + } + + private void consumed(long offset) { + if (!hasValidPosition) + throw new IllegalStateException("Cannot update consumed position without valid consumed/fetched positions"); + this.consumed = offset; + } + + private void committed(OffsetAndMetadata offset) { + this.committed = offset; + } + + private void pause() { + this.paused = true; + } + + private void resume() { + this.paused = false; + } + + private boolean isFetchable() { + return !paused && hasValidPosition; + } + + } + +} \ 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 f58b8508d3f81..d42fae9431063 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 @@ -3,9 +3,9 @@ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the * License. You may obtain a copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the * specific language governing permissions and limitations under the License. @@ -13,17 +13,16 @@ package org.apache.kafka.clients.producer; import java.net.InetSocketAddress; -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.ClientUtils; +import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.NetworkClient; -import org.apache.kafka.clients.producer.internals.Metadata; -import org.apache.kafka.clients.producer.internals.Partitioner; import org.apache.kafka.clients.producer.internals.RecordAccumulator; import org.apache.kafka.clients.producer.internals.Sender; import org.apache.kafka.common.Cluster; @@ -33,18 +32,23 @@ 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.network.ChannelBuilder; 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.utils.ClientUtils; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.common.utils.KafkaThread; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; @@ -54,18 +58,76 @@ /** * 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. *

    - * 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. + * 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 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 static final String JMX_PREFIX = "kafka.producer"; + private String clientId; private final Partitioner partitioner; private final int maxRequestSize; - private final long metadataFetchTimeoutMs; private final long totalMemorySize; private final Metadata metadata; private final RecordAccumulator accumulator; @@ -75,77 +137,188 @@ public class KafkaProducer implements Producer { private final CompressionType compressionType; private final Sensor errors; private final Time time; + private final Serializer keySerializer; + private final Serializer valueSerializer; + private final ProducerConfig producerConfig; + private final long maxBlockTimeMs; + private final int requestTimeoutMs; /** * 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); + } + + /** + * 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 KafkaProducer(ProducerConfig config) { - log.trace("Starting the Kafka producer"); - 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); - String jmxPrefix = "kafka.producer." + (clientId.length() > 0 ? clientId + "." : ""); - 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 = new Partitioner(); - 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)); - this.accumulator = new RecordAccumulator(config.getInt(ProducerConfig.BATCH_SIZE_CONFIG), - this.totalMemorySize, - config.getLong(ProducerConfig.LINGER_MS_CONFIG), - retryBackoffMs, - config.getBoolean(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG), - metrics, - time); - List addresses = ClientUtils.parseAndValidateAddresses(config.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)); - this.metadata.update(Cluster.bootstrap(addresses), time.milliseconds()); - - NetworkClient client = new NetworkClient(new Selector(this.metrics, time), - 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()); - this.ioThread = new KafkaThread("kafka-producer-network-thread", this.sender, true); - this.ioThread.start(); - - this.errors = this.metrics.sensor("errors"); - - config.logUnused(); - log.debug("Kafka producer started"); + @SuppressWarnings("unchecked") + private KafkaProducer(ProducerConfig config, Serializer keySerializer, Serializer valueSerializer) { + try { + log.trace("Starting the Kafka producer"); + Map userProvidedConfigs = config.originals(); + 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); + clientId = config.getString(ProducerConfig.CLIENT_ID_CONFIG); + if (clientId.length() <= 0) + clientId = "producer-" + PRODUCER_CLIENT_ID_SEQUENCE.getAndIncrement(); + List reporters = config.getConfiguredInstances(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG, + MetricsReporter.class); + reporters.add(new JmxReporter(JMX_PREFIX)); + 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.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)); + /* check for user defined settings. + * If the BLOCK_ON_BUFFER_FULL is set to true,we do not honor METADATA_FETCH_TIMEOUT_CONFIG. + * This should be removed with release 0.9 when the deprecated configs are removed. + */ + if (userProvidedConfigs.containsKey(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG)) { + log.warn(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG + " config is deprecated and will be removed soon. " + + "Please use " + ProducerConfig.MAX_BLOCK_MS_CONFIG); + boolean blockOnBufferFull = config.getBoolean(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG); + if (blockOnBufferFull) { + this.maxBlockTimeMs = Long.MAX_VALUE; + } else if (userProvidedConfigs.containsKey(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG)) { + log.warn(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG + " config is deprecated and will be removed soon. " + + "Please use " + ProducerConfig.MAX_BLOCK_MS_CONFIG); + this.maxBlockTimeMs = config.getLong(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG); + } else { + this.maxBlockTimeMs = config.getLong(ProducerConfig.MAX_BLOCK_MS_CONFIG); + } + } else if (userProvidedConfigs.containsKey(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG)) { + log.warn(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG + " config is deprecated and will be removed soon. " + + "Please use " + ProducerConfig.MAX_BLOCK_MS_CONFIG); + this.maxBlockTimeMs = config.getLong(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG); + } else { + this.maxBlockTimeMs = config.getLong(ProducerConfig.MAX_BLOCK_MS_CONFIG); + } + + /* check for user defined settings. + * If the TIME_OUT config is set use that for request timeout. + * This should be removed with release 0.9 + */ + if (userProvidedConfigs.containsKey(ProducerConfig.TIMEOUT_CONFIG)) { + log.warn(ProducerConfig.TIMEOUT_CONFIG + " config is deprecated and will be removed soon. Please use " + + ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG); + this.requestTimeoutMs = config.getInt(ProducerConfig.TIMEOUT_CONFIG); + } else { + this.requestTimeoutMs = config.getInt(ProducerConfig.REQUEST_TIMEOUT_MS_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, + metrics, + time, + metricTags); + List addresses = ClientUtils.parseAndValidateAddresses(config.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)); + this.metadata.update(Cluster.bootstrap(addresses), time.milliseconds()); + ChannelBuilder channelBuilder = ClientUtils.createChannelBuilder(config.values()); + NetworkClient client = new NetworkClient( + new Selector(config.getLong(ProducerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), this.metrics, time, "producer", metricTags, channelBuilder), + 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.requestTimeoutMs); + 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), + this.metrics, + new SystemTime(), + clientId, + this.requestTimeoutMs); + 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 { + config.ignore(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG); + this.keySerializer = keySerializer; + } + if (valueSerializer == null) { + this.valueSerializer = config.getConfiguredInstance(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + Serializer.class); + this.valueSerializer.configure(config.originals(), false); + } else { + config.ignore(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG); + this.valueSerializer = valueSerializer; + } + config.logUnused(); + AppInfoParser.registerAppInfo(JMX_PREFIX, clientId); + 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) { @@ -157,10 +330,11 @@ private static int parseAcks(String acksString) { } /** - * 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); } @@ -176,72 +350,99 @@ 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 { // first make sure the metadata for the topic is available - waitOnMetadata(record.topic(), this.metadataFetchTimeoutMs); - int partition = partitioner.partition(record, metadata.fetch()); - int serializedSize = Records.LOG_OVERHEAD + Record.recordSize(record.key(), record.value()); + long startTime = time.milliseconds(); + waitOnMetadata(record.topic(), this.maxBlockTimeMs); + 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"); + } + long remainingTime = checkMaybeGetRemainingTime(startTime); + 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"); + } + remainingTime = checkMaybeGetRemainingTime(startTime); + int partition = partition(record, serializedKey, serializedValue, metadata.fetch()); + remainingTime = checkMaybeGetRemainingTime(startTime); + int serializedSize = Records.LOG_OVERHEAD + Record.recordSize(serializedKey, serializedValue); ensureValidRecordSize(serializedSize); TopicPartition tp = new TopicPartition(record.topic(), partition); log.trace("Sending record {} with callback {} to topic {} partition {}", record, callback, record.topic(), partition); - RecordAccumulator.RecordAppendResult result = accumulator.append(tp, record.key(), record.value(), compressionType, callback); + remainingTime = checkMaybeGetRemainingTime(startTime); + RecordAccumulator.RecordAppendResult result = accumulator.append(tp, serializedKey, serializedValue, callback, remainingTime); 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, + // 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); @@ -251,7 +452,11 @@ public Future send(ProducerRecord record, Callback callback) { return new FutureFailure(e); } catch (InterruptedException e) { this.errors.record(); - throw new KafkaException(e); + 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; @@ -263,7 +468,11 @@ public Future send(ProducerRecord record, Callback callback) { * @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) { + 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 { @@ -272,7 +481,6 @@ private void waitOnMetadata(String topic, long maxWaitMs) { while (metadata.fetch().partitionsForTopic(topic) == null) { log.trace("Requesting metadata update for topic {}.", topic); int version = metadata.requestUpdate(); - metadata.add(topic); sender.wakeup(); metadata.awaitUpdate(version, remainingWaitMs); long elapsed = time.milliseconds() - begin; @@ -299,31 +507,193 @@ private void ensureValidRecordSize(int size) { " 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 + */ + @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) { - waitOnMetadata(topic, this.metadataFetchTimeoutMs); + try { + waitOnMetadata(topic, this.maxBlockTimeMs); + } 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() { - log.trace("Closing the Kafka producer."); - 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); + } + } + } } - this.metrics.close(); + + 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); + AppInfoParser.unregisterAppInfo(JMX_PREFIX, clientId); 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; + } + return this.partitioner.partition(record.topic(), record.key(), serializedKey, record.value(), serializedValue, + cluster); + } + + /** + * Check and may be get the time elapsed since startTime. + * Throws a {@link org.apache.kafka.common.errors.TimeoutException} if the elapsed time + * is more than the max time to block (max.block.ms) + * + * @param startTime timestamp used to check the elapsed time + * @return remainingTime + */ + private long checkMaybeGetRemainingTime(long startTime) { + long elapsedTime = time.milliseconds() - startTime; + if (elapsedTime > maxBlockTimeMs) { + throw new TimeoutException("Request timed out"); + } + long remainingTime = maxBlockTimeMs - elapsedTime; + + return remainingTime; } 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 c0f1d57e0feb8..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,20 +97,20 @@ 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.partitionsForTopic(record.topic()) != null) - partition = partitioner.partition(record, this.cluster); + partition = partition(record, this.cluster); ProduceRequestResult result = new ProduceRequestResult(); FutureRecordMetadata future = new FutureRecordMetadata(result, 0); TopicPartition topicPartition = new TopicPartition(record.topic(), partition); @@ -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.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 f9de4af426449..c3dbd10479582 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 @@ -3,9 +3,9 @@ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the * License. You may obtain a copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the * specific language governing permissions and limitations under the License. @@ -14,17 +14,23 @@ 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.SSLConfigs; 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; /** * Configuration for the Kafka Producer. Documentation for these configurations can be found in the Kafka documentation + * href="http://kafka.apache.org/documentation.html#newproducerconfigs">Kafka documentation */ public class ProducerConfig extends AbstractConfig { @@ -33,25 +39,23 @@ public class ProducerConfig extends AbstractConfig { * CHANGE WILL BREAK USER CODE. */ - private static final ConfigDef config; + private static final ConfigDef CONFIG; /** bootstrap.servers */ - public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers"; - private static final String BOOSTRAP_SERVERS_DOC = "A list of host/port pairs to use for establishing the initial connection to the Kafka cluster. Data will be load " + "balanced over 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). If no server in this list is available sending " - + "data will fail until on becomes available."; + public static final String BOOTSTRAP_SERVERS_CONFIG = CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; /** metadata.fetch.timeout.ms */ + /** + * @deprecated This config will be removed in a future release. Please use {@link #MAX_BLOCK_MS_CONFIG} + */ + @Deprecated 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 configuration controls the maximum amount of time we will block waiting for the metadata " + 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."; /** metadata.max.age.ms */ - public static final String METADATA_MAX_AGE_CONFIG = "metadata.max.age.ms"; - private 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 METADATA_MAX_AGE_CONFIG = CommonClientConfigs.METADATA_MAX_AGE_CONFIG; + private static final String METADATA_MAX_AGE_DOC = CommonClientConfigs.METADATA_MAX_AGE_DOC; /** batch.size */ public static final String BATCH_SIZE_CONFIG = "batch.size"; @@ -77,7 +81,8 @@ public class ProducerConfig extends AbstractConfig { /** 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: " + 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" @@ -89,11 +94,14 @@ public class ProducerConfig extends AbstractConfig { + " 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." - + "
    • Other settings such as acks=2 are also possible, and will require the given number of" - + " acknowledgements but this is generally less useful."; + + " remains alive. This is the strongest available guarantee."; /** timeout.ms */ + + /** + * @deprecated This config will be removed in a future release. Please use {@link #REQUEST_TIMEOUT_MS_CONFIG} + */ + @Deprecated 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 " @@ -112,17 +120,13 @@ public class ProducerConfig extends AbstractConfig { + "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."; /** client.id */ - public static final String CLIENT_ID_CONFIG = "client.id"; - private static final String CLIENT_ID_DOC = "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 with the request. The " - + "application can set any string it wants as this has no functional purpose other than in logging and metrics."; + public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG; /** send.buffer.bytes */ - public static final String SEND_BUFFER_CONFIG = "send.buffer.bytes"; - private static final String SEND_BUFFER_DOC = "The size of the TCP send buffer to use when sending data"; + public static final String SEND_BUFFER_CONFIG = CommonClientConfigs.SEND_BUFFER_CONFIG; /** receive.buffer.bytes */ - public static final String RECEIVE_BUFFER_CONFIG = "receive.buffer.bytes"; - private static final String RECEIVE_BUFFER_DOC = "The size of the TCP receive buffer to use when reading data"; + public static final String RECEIVE_BUFFER_CONFIG = CommonClientConfigs.RECEIVE_BUFFER_CONFIG; /** max.request.size */ public static final String MAX_REQUEST_SIZE_CONFIG = "max.request.size"; @@ -130,10 +134,13 @@ public class ProducerConfig extends AbstractConfig { + "batches the producer will send in a single request to avoid sending huge requests."; /** reconnect.backoff.ms */ - public static final String RECONNECT_BACKOFF_MS_CONFIG = "reconnect.backoff.ms"; - private static final String RECONNECT_BACKOFF_MS_DOC = "The amount of time to wait before attempting to reconnect to a given host when a connection fails." + " This avoids a scenario where the client repeatedly attempts to connect to a host in a tight loop."; + public static final String RECONNECT_BACKOFF_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG; /** block.on.buffer.full */ + /** + * @deprecated This config will be removed in a future release. Also, the {@link #METADATA_FETCH_TIMEOUT_CONFIG} is no longer honored when this property is set to true. + */ + @Deprecated 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."; @@ -146,81 +153,166 @@ public class ProducerConfig extends AbstractConfig { + "may appear first."; /** retry.backoff.ms */ - public static final String RETRY_BACKOFF_MS_CONFIG = "retry.backoff.ms"; - private static final String RETRY_BACKOFF_MS_DOC = "The amount of time to wait before attempting to retry a failed produce request to a given topic partition." + " This avoids repeated sending-and-failing in a tight loop."; + 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, or snappy. Compression is of full batches of data, " - + " so the efficacy of batching will also impact the compression ratio (more batching means better compression)."; + 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 = "metrics.sample.window.ms"; - private static final String METRICS_SAMPLE_WINDOW_MS_DOC = "The metrics system maintains a configurable number of samples over a fixed window size. This configuration " + "controls the size of the window. For example we might maintain two samples each measured over a 30 second period. " - + "When a window expires we erase and overwrite the oldest window."; + 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 = "metrics.num.samples"; - private static final String METRICS_NUM_SAMPLES_DOC = "The number of samples maintained to compute metrics."; + public static final String METRICS_NUM_SAMPLES_CONFIG = CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG; /** metric.reporters */ - 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. 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 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."; + 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"; + public static final String KEY_SERIALIZER_CLASS_DOC = "Serializer class for key that implements the Serializer interface."; + + /** value.serializer */ + public static final String VALUE_SERIALIZER_CLASS_CONFIG = "value.serializer"; + public 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."; + + /** max.block.ms */ + public static final String MAX_BLOCK_MS_CONFIG = "max.block.ms"; + private static final String MAX_BLOCK_MS_DOC = "The configuration controls how long {@link KafkaProducer#send()} and {@link KafkaProducer#partitionsFor} will block." + + "These methods can be blocked for multiple reasons. For e.g: buffer full, metadata unavailable." + + "This configuration imposes maximum limit on the total time spent in fetching metadata, serialization of key and value, partitioning and " + + "allocation of buffer memory when doing a send(). In case of partitionsFor(), this configuration imposes a maximum time threshold on waiting " + + "for metadata"; + + /** request.timeout.ms */ + public static final String REQUEST_TIMEOUT_MS_CONFIG = CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG; + private static final String REQUEST_TIMEOUT_MS_DOC = CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC; static { - config = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, BOOSTRAP_SERVERS_DOC) + 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", Importance.HIGH, ACKS_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, CLIENT_ID_DOC) - .define(SEND_BUFFER_CONFIG, Type.INT, 128 * 1024, atLeast(0), Importance.MEDIUM, SEND_BUFFER_DOC) - .define(RECEIVE_BUFFER_CONFIG, Type.INT, 32 * 1024, atLeast(0), Importance.MEDIUM, RECEIVE_BUFFER_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, 10L, atLeast(0L), Importance.LOW, RECONNECT_BACKOFF_MS_DOC) - .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, "", Importance.LOW, METRIC_REPORTER_CLASSES_DOC) - .define(RETRY_BACKOFF_MS_CONFIG, Type.LONG, 100L, atLeast(0L), Importance.LOW, RETRY_BACKOFF_MS_DOC) + .define(BLOCK_ON_BUFFER_FULL_CONFIG, Type.BOOLEAN, false, 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(MAX_BLOCK_MS_CONFIG, + Type.LONG, + 60 * 1000, + atLeast(0), + Importance.MEDIUM, + MAX_BLOCK_MS_DOC) + .define(REQUEST_TIMEOUT_MS_CONFIG, + Type.INT, + 30 * 1000, + atLeast(0), + Importance.MEDIUM, + REQUEST_TIMEOUT_MS_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, - METRICS_SAMPLE_WINDOW_MS_DOC) - .define(METRICS_NUM_SAMPLES_CONFIG, Type.INT, 2, atLeast(1), Importance.LOW, METRICS_NUM_SAMPLES_DOC) + 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); + 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) + .define(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, Type.STRING, CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL, Importance.MEDIUM, CommonClientConfigs.SECURITY_PROTOCOL_DOC) + .define(SSLConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, Type.CLASS, SSLConfigs.DEFAULT_PRINCIPAL_BUILDER_CLASS, Importance.LOW, SSLConfigs.PRINCIPAL_BUILDER_CLASS_DOC) + .define(SSLConfigs.SSL_PROTOCOL_CONFIG, Type.STRING, SSLConfigs.DEFAULT_SSL_PROTOCOL, Importance.MEDIUM, SSLConfigs.SSL_PROTOCOL_DOC) + .define(SSLConfigs.SSL_PROVIDER_CONFIG, Type.STRING, Importance.MEDIUM, SSLConfigs.SSL_PROVIDER_DOC, false) + .define(SSLConfigs.SSL_CIPHER_SUITES_CONFIG, Type.LIST, Importance.LOW, SSLConfigs.SSL_CIPHER_SUITES_DOC, false) + .define(SSLConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Type.LIST, SSLConfigs.DEFAULT_ENABLED_PROTOCOLS, Importance.MEDIUM, SSLConfigs.SSL_ENABLED_PROTOCOLS_DOC) + .define(SSLConfigs.SSL_KEYSTORE_TYPE_CONFIG, Type.STRING, SSLConfigs.DEFAULT_SSL_KEYSTORE_TYPE, Importance.MEDIUM, SSLConfigs.SSL_KEYSTORE_TYPE_DOC) + .define(SSLConfigs.SSL_KEYSTORE_LOCATION_CONFIG, Type.STRING, Importance.HIGH, SSLConfigs.SSL_KEYSTORE_LOCATION_DOC, false) + .define(SSLConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, Type.STRING, Importance.HIGH, SSLConfigs.SSL_KEYSTORE_PASSWORD_DOC, false) + .define(SSLConfigs.SSL_KEY_PASSWORD_CONFIG, Type.STRING, Importance.HIGH, SSLConfigs.SSL_KEY_PASSWORD_DOC, false) + .define(SSLConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, Type.STRING, SSLConfigs.DEFAULT_SSL_TRUSTSTORE_TYPE, Importance.MEDIUM, SSLConfigs.SSL_TRUSTSTORE_TYPE_DOC) + .define(SSLConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, Type.STRING, SSLConfigs.DEFAULT_TRUSTSTORE_LOCATION, Importance.HIGH, SSLConfigs.SSL_TRUSTSTORE_LOCATION_DOC) + .define(SSLConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, Type.STRING, SSLConfigs.DEFAULT_TRUSTSTORE_PASSWORD, Importance.HIGH, SSLConfigs.SSL_TRUSTSTORE_PASSWORD_DOC) + .define(SSLConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, Type.STRING, SSLConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM, Importance.LOW, SSLConfigs.SSL_KEYMANAGER_ALGORITHM_DOC) + .define(SSLConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, Type.STRING, SSLConfigs.DEFAULT_SSL_TRUSTMANAGER_ALGORITHM, Importance.LOW, SSLConfigs.SSL_TRUSTMANAGER_ALGORITHM_DOC) + .define(SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, Type.STRING, Importance.LOW, SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DOC, false) + /* 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()); + 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 c3181b368b6cf..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 @@ -20,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 @@ -35,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; @@ -51,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); } @@ -61,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); } @@ -75,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; } @@ -95,8 +95,38 @@ public Integer partition() { @Override public String toString() { - String key = this.key == null ? "null" : ("byte[" + this.key.length + "]"); - String value = this.value == null ? "null" : ("byte[" + this.value.length + "]"); + 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/internals/BufferPool.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java index 169a6568bf7ef..2a45075189466 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 @@ -16,19 +16,21 @@ */ package org.apache.kafka.clients.producer.internals; -import org.apache.kafka.clients.producer.BufferExhaustedException; -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; - 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.common.MetricName; +import org.apache.kafka.common.errors.TimeoutException; +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; + /** * A pool of ByteBuffers kept under a given memory limit. This class is fairly specific to the needs of the producer. In @@ -44,7 +46,6 @@ public final class BufferPool { private final long totalMemory; private final int poolableSize; - private final boolean blockOnExhaustion; private final ReentrantLock lock; private final Deque free; private final Deque waiters; @@ -58,13 +59,13 @@ public final class BufferPool { * * @param memory The maximum amount of memory that this buffer pool can allocate * @param poolableSize The buffer size to cache in the free list rather than deallocating - * @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, Metrics metrics, Time time) { + public BufferPool(long memory, int poolableSize, Metrics metrics, Time time , String metricGrpName , Map metricTags) { this.poolableSize = poolableSize; - this.blockOnExhaustion = blockOnExhaustion; this.lock = new ReentrantLock(); this.free = new ArrayDeque(); this.waiters = new ArrayDeque(); @@ -73,23 +74,25 @@ public BufferPool(long memory, int poolableSize, boolean blockOnExhaustion, Metr this.metrics = metrics; this.time = time; this.waitTime = this.metrics.sensor("bufferpool-wait-time"); - this.waitTime.add("bufferpool-wait-ratio", - "The fraction of time an appender waits for space allocation.", - new Rate(TimeUnit.NANOSECONDS)); - } + 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. 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 + * @param maxTimeToBlock The maximum time in milliseconds to block for buffer memory to be available * @return The buffer * @throws InterruptedException If the thread is interrupted while blocked * @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 */ - public ByteBuffer allocate(int size) throws InterruptedException { + public ByteBuffer allocate(int size, long maxTimeToBlock) throws InterruptedException { if (size > this.totalMemory) throw new IllegalArgumentException("Attempt to allocate " + size + " bytes, but there is a hard limit of " @@ -112,10 +115,6 @@ public ByteBuffer allocate(int size) throws InterruptedException { this.availableMemory -= size; lock.unlock(); return ByteBuffer.allocate(size); - } else if (!blockOnExhaustion) { - throw new BufferExhaustedException("You have exhausted the " + this.totalMemory - + " bytes of memory you configured for the client and the client is configured to error" - + " rather than block when memory is exhausted."); } else { // we are out of memory and will have to block int accumulated = 0; @@ -125,14 +124,12 @@ 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) { - try { - long startWait = time.nanoseconds(); - moreMemory.await(300, TimeUnit.MILLISECONDS); - long endWait = time.nanoseconds(); - this.waitTime.record(endWait - startWait, time.milliseconds()); - } catch (InterruptedException e) { - // This should never happen. Just let it go. - } + long startWait = time.nanoseconds(); + if (!moreMemory.await(maxTimeToBlock, TimeUnit.MILLISECONDS)) + throw new TimeoutException("Failed to allocate memory within the configured max blocking time"); + 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()) { 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 index 678d1c6accd2a..747e29fadbcf6 100644 --- 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 @@ -23,12 +23,18 @@ public class ErrorLoggingCallback implements Callback { 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; - this.value = value; + + if (logAsString) { + this.value = value; + } + + this.valueLength = value == null ? -1 : value.length; this.logAsString = logAsString; } @@ -36,10 +42,10 @@ public void onCompletion(RecordMetadata metadata, Exception e) { if (e != null) { String keyString = (key == null) ? "null" : logAsString ? new String(key) : key.length + " bytes"; - String valueString = (value == null) ? "null" : - logAsString ? new String(value) : value.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()); + 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 4a2da41f47994..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 @@ -51,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/Partitioner.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java deleted file mode 100644 index 40e8234f87710..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 cluster The current cluster metadata - */ - public int partition(ProducerRecord record, Cluster cluster) { - List partitions = cluster.partitionsForTopic(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 c5d470011d334..eed2a5ef53f6b 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 @@ -12,26 +12,18 @@ */ package org.apache.kafka.clients.producer.internals; -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.Iterator; 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; @@ -42,6 +34,19 @@ 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. @@ -55,68 +60,85 @@ public final class RecordAccumulator { 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 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, + CompressionType compression, long lingerMs, long retryBackoffMs, - boolean blockOnBufferFull, Metrics metrics, - Time time) { + 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, metrics, time); + String metricGrpName = "producer-metrics"; + this.free = new BufferPool(totalSize, batchSize, metrics, time , metricGrpName , metricTags); + this.incomplete = new IncompleteRecordBatches(); this.time = time; - registerMetrics(metrics); + registerMetrics(metrics, metricGrpName, metricTags); } - private void registerMetrics(Metrics metrics) { - metrics.addMetric("waiting-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 maximum amount of buffer memory the client can use (whether or not it is currently used).", - 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 not being used (either unallocated or in the free list).", - new Measurable() { - public double measure(MetricConfig config, long now) { - return free.availableMemory(); - } - }); + 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()); } /** @@ -128,46 +150,93 @@ public double measure(MetricConfig config, long now) { * @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 + * @param maxTimeToBlock The maximum time in milliseconds to block for buffer memory to be available */ - public RecordAppendResult 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 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); + public RecordAppendResult append(TopicPartition tp, byte[] key, byte[] value, Callback callback, long maxTimeToBlock) 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, time.milliseconds()); + 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)); - log.trace("Allocating a new {} byte message buffer for topic {} partition {}", size, tp.topic(), tp.partition()); - ByteBuffer buffer = free.allocate(size); - synchronized (dq) { - 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); + // 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, maxTimeToBlock); + 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, time.milliseconds()); + 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, time.milliseconds())); + + dq.addLast(batch); + incomplete.add(batch); + return new RecordAppendResult(future, dq.size() > 1 || batch.records.isFull(), true); } - 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)); + } finally { + appendsInProgress.decrementAndGet(); + } + } - dq.addLast(batch); - return new RecordAppendResult(future, dq.size() > 1 || batch.records.isFull(), true); + /** + * Abort the batches that have been sitting in RecordAccumulator for more than the configured requestTimeout + * due to metadata being unavailable + */ + public List abortExpiredBatches(int requestTimeout, Cluster cluster, long now) { + List expiredBatches = new ArrayList(); + int count = 0; + for (Map.Entry> entry : this.batches.entrySet()) { + TopicPartition topicAndPartition = entry.getKey(); + Deque dq = entry.getValue(); + synchronized (dq) { + // iterate over the batches and expire them if they have stayed in accumulator for more than requestTimeOut + Iterator batchIterator = dq.iterator(); + while (batchIterator.hasNext()) { + RecordBatch batch = batchIterator.next(); + Node leader = cluster.leaderFor(topicAndPartition); + if (leader == null) { + // check if the batch is expired + if (batch.maybeExpire(requestTimeout, now, this.lingerMs)) { + expiredBatches.add(batch); + count++; + batchIterator.remove(); + deallocate(batch); + } else { + if (!batch.inRetry()) { + break; + } + } + } + } + } } + if (expiredBatches.size() > 0) + log.trace("Expired {} batches in accumulator", count); + + return expiredBatches; } /** @@ -176,6 +245,8 @@ public RecordAppendResult append(TopicPartition tp, byte[] key, byte[] value, Co public void reenqueue(RecordBatch batch, long now) { batch.attempts++; batch.lastAttemptMs = now; + batch.lastAppendTime = now; + batch.setRetry(); Deque deque = dequeFor(batch.topicPartition); synchronized (deque) { deque.addFirst(batch); @@ -183,9 +254,9 @@ public void reenqueue(RecordBatch batch, long now) { } /** - * Get a list of nodes whose partitions are ready to be sent, and the time to when any partition will be ready if no - * partitions are ready yet; If the ready nodes list is non-empty, the timeout value will be 0. Also return the flag - * for whether there are any unknown leaders for the accumulated partition batches. + * 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 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 : @@ -219,11 +290,16 @@ public ReadyCheckResult ready(Cluster cluster, long nowMs) { long timeToWaitMs = backingOff ? retryBackoffMs : lingerMs; long timeLeftMs = Math.max(timeToWaitMs - waitedTimeMs, 0); boolean full = deque.size() > 1 || batch.records.isFull(); - boolean expired = waitedTimeMs >= lingerMs; - boolean sendable = full || expired || exhausted || closed; - if (sendable && !backingOff) + boolean expired = waitedTimeMs >= timeToWaitMs; + boolean sendable = full || expired || exhausted || closed || flushInProgress(); + if (sendable && !backingOff) { readyNodes.add(leader); - nextReadyCheckDelayMs = Math.min(timeLeftMs, nextReadyCheckDelayMs); + } 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); + } } } } @@ -255,7 +331,6 @@ public boolean hasUnsent() { * @param maxSize The maximum number of bytes to drain * @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. - * TODO: There may be a starvation issue due to iteration order */ public Map> drain(Cluster cluster, Set nodes, int maxSize, long now) { if (nodes.isEmpty()) @@ -275,17 +350,21 @@ public Map> drain(Cluster cluster, Set nodes, i synchronized (deque) { RecordBatch first = deque.peekFirst(); if (first != null) { - 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; + 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; + } } } } @@ -313,8 +392,73 @@ private Deque dequeFor(TopicPartition tp) { * 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); + } + } /** * Close this accumulator and force all the record buffers to be drained @@ -323,7 +467,9 @@ 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; @@ -336,6 +482,9 @@ public RecordAppendResult(FutureRecordMetadata future, boolean batchIsFull, bool } } + /* + * 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; @@ -347,4 +496,35 @@ public ReadyCheckResult(Set readyNodes, long nextReadyCheckDelayMs, boolea 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 dd0af8aee98ab..3f185829487ec 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 @@ -16,7 +16,9 @@ import java.util.List; 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.errors.TimeoutException; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Record; import org.slf4j.Logger; @@ -39,8 +41,10 @@ public final class RecordBatch { public long lastAttemptMs; public final MemoryRecords records; public final TopicPartition topicPartition; - private final ProduceRequestResult produceFuture; + public final ProduceRequestResult produceFuture; + public long lastAppendTime; private final List thunks; + private boolean retry; public RecordBatch(TopicPartition tp, MemoryRecords records, long now) { this.createdMs = now; @@ -49,6 +53,8 @@ public RecordBatch(TopicPartition tp, MemoryRecords records, long now) { this.topicPartition = tp; this.produceFuture = new ProduceRequestResult(); this.thunks = new ArrayList(); + this.lastAppendTime = createdMs; + this.retry = false; } /** @@ -56,12 +62,13 @@ 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, Callback callback) { + public FutureRecordMetadata tryAppend(byte[] key, byte[] value, Callback callback, long now) { if (!this.records.hasRoomFor(key, value)) { return null; } else { this.records.append(0L, key, value); this.maxRecordSize = Math.max(this.maxRecordSize, Record.recordSize(key, value)); + this.lastAppendTime = now; FutureRecordMetadata future = new FutureRecordMetadata(this.produceFuture, this.recordCount); if (callback != null) thunks.add(new Thunk(callback, future)); @@ -77,7 +84,6 @@ public FutureRecordMetadata tryAppend(byte[] key, byte[] value, Callback callbac * @param exception The exception that occurred (or null if the request was successful) */ public void done(long baseOffset, RuntimeException exception) { - this.produceFuture.done(topicPartition, baseOffset, exception); log.trace("Produced messages to topic-partition {} with base offset offset {} and error: {}.", topicPartition, baseOffset, @@ -86,14 +92,17 @@ public void done(long baseOffset, RuntimeException exception) { for (int i = 0; i < this.thunks.size(); i++) { try { Thunk thunk = this.thunks.get(i); - if (exception == null) - thunk.callback.onCompletion(thunk.future.get(), 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) { log.error("Error executing user-provided callback on message for topic-partition {}:", topicPartition, e); } } + this.produceFuture.done(topicPartition, baseOffset, exception); } /** @@ -113,4 +122,34 @@ public Thunk(Callback callback, FutureRecordMetadata future) { public String toString() { return "RecordBatch(topicPartition=" + topicPartition + ", recordCount=" + recordCount + ")"; } + + /** + * Expire the batch that is ready but is sitting in accumulator for more than requestTimeout due to metadata being unavailable. + * We need to explicitly check if the record is full or linger time is met because the accumulator's partition may not be ready + * if the leader is unavailable. + */ + public boolean maybeExpire(int requestTimeout, long now, long lingerMs) { + boolean expire = false; + if ((this.records.isFull() && requestTimeout < (now - this.lastAppendTime)) || requestTimeout < (now - (this.lastAttemptMs + lingerMs))) { + expire = true; + this.records.close(); + this.done(-1L, new TimeoutException("Batch Expired")); + } + + return expire; + } + + /** + * Returns if the batch is been retried for sending to kafka + */ + public boolean inRetry() { + return this.retry; + } + + /** + * Set retry to true if the batch is being retried (for send) + */ + public void setRetry() { + this.retry = true; + } } \ 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 8ebe7ed82c938..134d45a240d61 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 @@ -16,12 +16,15 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; 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; @@ -29,6 +32,7 @@ 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; @@ -67,9 +71,6 @@ public class Sender implements Runnable { /* 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; - /* the number of times to retry a failed request before giving up */ private final int retries; @@ -79,28 +80,39 @@ public class Sender implements Runnable { /* true while the sender thread is still running */ private volatile boolean running; + /* 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; + + /* the max time to wait for the server to respond to the request*/ + private final int requestTimeout; + public Sender(KafkaClient client, Metadata metadata, RecordAccumulator accumulator, int maxRequestSize, short acks, int retries, - int requestTimeout, Metrics metrics, - Time time) { + Time time, + String clientId, + int requestTimeout) { this.client = client; this.accumulator = accumulator; this.metadata = metadata; this.maxRequestSize = maxRequestSize; this.running = true; - this.requestTimeout = requestTimeout; this.acks = acks; this.retries = retries; this.time = time; + this.clientId = clientId; this.sensors = new SenderMetrics(metrics); + this.requestTimeout = requestTimeout; } /** @@ -123,15 +135,23 @@ public void run() { // okay we stopped accepting requests but there may still be // requests in the accumulator or waiting for acknowledgment, // wait until these are completed. - while (this.accumulator.hasUnsent() || this.client.inFlightRequestCount() > 0) { + while (!forceClose && (this.accumulator.hasUnsent() || this.client.inFlightRequestCount() > 0)) { try { run(time.milliseconds()); } catch (Exception e) { log.error("Uncaught error in kafka producer I/O thread: ", e); } } - - this.client.close(); + 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); + } log.debug("Shutdown of Kafka producer I/O thread has completed."); } @@ -139,7 +159,8 @@ public void run() { /** * Run a single iteration of sending * - * @param now The current POSIX time in milliseconds + * @param now + * The current POSIX time in milliseconds */ public void run(long now) { Cluster cluster = metadata.fetch(); @@ -152,33 +173,46 @@ public void run(long now) { // 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)) + if (!this.client.ready(node, now)) { iter.remove(); + notReadyTimeout = Math.min(notReadyTimeout, this.client.connectionDelay(node, now)); + } } // create produce requests - Map> batches = this.accumulator.drain(cluster, result.readyNodes, this.maxRequestSize, now); - List requests = createProduceRequests(batches, now); - sensors.updateProduceRequestMetrics(requests); + Map> batches = this.accumulator.drain(cluster, + result.readyNodes, + this.maxRequestSize, + now); + + List expiredBatches = this.accumulator.abortExpiredBatches(this.requestTimeout, cluster, now); + // update sensors + for (RecordBatch expiredBatch : expiredBatches) + this.sensors.recordErrors(expiredBatch.topicPartition.topic(), expiredBatch.recordCount); + 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, now); // 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; - List responses = this.client.poll(requests, result.nextReadyCheckDelayMs, now); - for (ClientResponse response : responses) { - if (response.wasDisconnected()) - handleDisconnect(response, now); - else - handleResponse(response, now); - } + this.client.poll(pollTimeout, now); } /** @@ -190,45 +224,54 @@ public void initiateClose() { this.wakeup(); } - private void handleDisconnect(ClientResponse response, long now) { - log.trace("Cancelled request {} due to node {} being disconnected", response, response.request().request().destination()); - int correlation = response.request().request().header().correlationId(); - @SuppressWarnings("unchecked") - Map responseBatches = (Map) response.request().attachment(); - for (RecordBatch batch : responseBatches.values()) - completeBatch(batch, Errors.NETWORK_EXCEPTION, -1L, correlation, now); + /** + * Closes the sender without sending out any pending messages. + */ + public void forceClose() { + this.forceClose = true; + initiateClose(); } /** * Handle a produce response */ - private void handleResponse(ClientResponse response, long now) { + private void handleProduceResponse(ClientResponse response, Map batches, long now) { int correlationId = response.request().request().header().correlationId(); - log.trace("Received produce response from node {} with correlation id {}", - response.request().request().destination(), - correlationId); - @SuppressWarnings("unchecked") - Map batches = (Map) response.request().attachment(); - // 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 + 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.NONE, -1L, correlationId, now); + 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()); + this.sensors.recordThrottleTime(response.request().request().destination(), + produceResponse.getThrottleTime()); + } else { + // this is the acks = 0 case, just complete all requests + for (RecordBatch batch : batches.values()) + completeBatch(batch, Errors.NONE, -1L, correlationId, now); + } } } /** * 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 @@ -278,17 +321,22 @@ private List createProduceRequests(Map */ private ClientRequest produceRequest(long now, int destination, short acks, int timeout, List batches) { Map produceRecordsByPartition = new HashMap(batches.size()); - Map recordsByPartition = new HashMap(batches.size()); + final Map recordsByPartition = new HashMap(batches.size()); for (RecordBatch batch : batches) { TopicPartition tp = batch.topicPartition; - ByteBuffer recordsBuffer = batch.records.buffer(); - recordsBuffer.flip(); - produceRecordsByPartition.put(tp, recordsBuffer); + produceRecordsByPartition.put(tp, (ByteBuffer) batch.records.buffer().flip()); recordsByPartition.put(tp, batch); } ProduceRequest request = new ProduceRequest(acks, timeout, produceRecordsByPartition); - RequestSend send = new RequestSend(destination, this.client.nextRequestHeader(ApiKeys.PRODUCE), request.toStruct()); - return new ClientRequest(now, acks != 0, send, recordsByPartition); + 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()); + } + }; + return new ClientRequest(now, acks != 0, send, callback); } /** @@ -312,51 +360,72 @@ private class SenderMetrics { public final Sensor batchSizeSensor; public final Sensor compressionRateSensor; public final Sensor maxRecordSizeSensor; + public final Sensor produceThrottleTimeSensor; 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"); - this.batchSizeSensor.add("batch-size-avg", "The average number of bytes sent per partition per-request.", new Avg()); - this.batchSizeSensor.add("batch-size-max", "The max number of bytes sent per partition per-request.", new Max()); + 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"); - this.compressionRateSensor.add("compression-rate-avg", "The average compression rate of record batches.", new Avg()); + 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"); - this.queueTimeSensor.add("record-queue-time-avg", - "The average time in ms record batches spent in the record accumulator.", - new Avg()); - this.queueTimeSensor.add("record-queue-time-max", - "The maximum time in ms record batches spent in the record accumulator.", - new Max()); + 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"); - this.requestTimeSensor.add("request-latency-avg", "The average request latency in ms", new Avg()); - this.requestTimeSensor.add("request-latency-max", "The maximum request latency in ms", new Max()); + 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.produceThrottleTimeSensor = metrics.sensor("produce-throttle-time"); + m = new MetricName("produce-throttle-time-avg", metricGrpName, "The average throttle time in ms", metricTags); + this.produceThrottleTimeSensor.add(m, new Avg()); + m = new MetricName("produce-throttle-time-max", metricGrpName, "The maximum throttle time in ms", metricTags); + this.produceThrottleTimeSensor.add(m, new Max()); this.recordsPerRequestSensor = metrics.sensor("records-per-request"); - this.recordsPerRequestSensor.add("record-send-rate", "The average number of records sent per second.", new Rate()); - this.recordsPerRequestSensor.add("records-per-request-avg", "The average number of records per request.", new Avg()); + 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"); - this.retrySensor.add("record-retry-rate", "The average per-second number of retried record sends", new Rate()); + 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"); - this.errorSensor.add("record-error-rate", "The average per-second number of record sends that resulted in errors", new Rate()); + 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"); - this.maxRecordSizeSensor.add("record-size-max", "The maximum record size", new Max()); - this.maxRecordSizeSensor.add("record-size-avg", "The average record size", new Avg()); + 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()); - this.metrics.addMetric("requests-in-flight", "The current number of in-flight requests awaiting a response.", new Measurable() { + 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(); } }); - metrics.addMetric("metadata-age", "The age in seconds of the current producer metadata being used.", new Measurable() { + 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.lastUpdate()) / 1000.0; + return (now - metadata.lastSuccessfulUpdate()) / 1000.0; } }); } @@ -367,65 +436,69 @@ public void maybeRegisterTopicMetrics(String topic) { 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); - topicRecordCount.add("topic." + topic + ".record-send-rate", new Rate()); + 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); - topicByteRate.add("topic." + topic + ".byte-rate", new Rate()); + m = new MetricName("byte-rate", metricGrpName , metricTags); + topicByteRate.add(m, new Rate()); String topicCompressionRateName = "topic." + topic + ".compression-rate"; Sensor topicCompressionRate = this.metrics.sensor(topicCompressionRateName); - topicCompressionRate.add("topic." + topic + ".compression-rate", new Avg()); + m = new MetricName("compression-rate", metricGrpName , metricTags); + topicCompressionRate.add(m, new Avg()); String topicRetryName = "topic." + topic + ".record-retries"; Sensor topicRetrySensor = this.metrics.sensor(topicRetryName); - topicRetrySensor.add("topic." + topic + ".record-retry-rate", new Rate()); + 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); - topicErrorSensor.add("topic." + topic + ".record-error-rate", new Rate()); + m = new MetricName("record-error-rate", metricGrpName , metricTags); + topicErrorSensor.add(m, new Rate()); } } - public void updateProduceRequestMetrics(List requests) { + public void updateProduceRequestMetrics(Map> batches) { long now = time.milliseconds(); - for (int i = 0; i < requests.size(); i++) { - ClientRequest request = requests.get(i); + for (List nodeBatch : batches.values()) { int records = 0; - - if (request.attachment() != null) { - Map responseBatches = (Map) request.attachment(); - for (RecordBatch batch : responseBatches.values()) { - - // 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); + 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); } } @@ -447,16 +520,21 @@ public void recordErrors(String topic, int count) { topicErrorSensor.record(count, now); } - public void recordLatency(int node, long latency) { + public void recordLatency(String node, long latency) { long now = time.milliseconds(); this.requestTimeSensor.record(latency, now); - if (node >= 0) { + if (!node.isEmpty()) { String nodeTimeName = "node-" + node + ".latency"; Sensor nodeRequestTime = this.metrics.getSensor(nodeTimeName); if (nodeRequestTime != null) nodeRequestTime.record(latency, now); } } + + public void recordThrottleTime(String node, long throttleTimeMs) { + this.produceThrottleTimeSensor.record(throttleTimeMs, time.milliseconds()); + } + } } 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 d3299b944062d..60594a7dce901 100644 --- a/clients/src/main/java/org/apache/kafka/common/Cluster.java +++ b/clients/src/main/java/org/apache/kafka/common/Cluster.java @@ -25,7 +25,9 @@ public final class Cluster { 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 @@ -37,6 +39,10 @@ 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()); @@ -63,8 +69,18 @@ public Cluster(Collection nodes, Collection partitions) { } } this.partitionsByTopic = new HashMap>(partsForTopic.size()); - for (Map.Entry> entry : partsForTopic.entrySet()) - this.partitionsByTopic.put(entry.getKey(), Collections.unmodifiableList(entry.getValue())); + 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.partitionsByNode = new HashMap>(partsForNode.size()); for (Map.Entry> entry : partsForNode.entrySet()) this.partitionsByNode.put(entry.getKey(), Collections.unmodifiableList(entry.getValue())); @@ -97,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 @@ -129,6 +154,15 @@ public List partitionsForTopic(String topic) { return this.partitionsByTopic.get(topic); } + /** + * 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 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..ee50f332d3e77 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/MetricName.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.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 pairs"); + Map tags = new HashMap(); + + for (int i = 0; i < keyValue.length; i += 2) + 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 0e47ff3ff0e05..644cd71c8cb4f 100644 --- a/clients/src/main/java/org/apache/kafka/common/Node.java +++ b/clients/src/main/java/org/apache/kafka/common/Node.java @@ -18,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 */ @@ -35,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 */ @@ -82,7 +96,7 @@ public boolean equals(Object obj) { @Override public String toString() { - return "Node(" + (id < 0 ? "" : id + ", ") + host + ", " + port + ")"; + return "Node(" + id + ", " + host + ", " + port + ")"; } } 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 b15aa2c3ef2d7..321da8afc7394 100644 --- a/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java +++ b/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java @@ -69,10 +69,10 @@ public Node[] inSyncReplicas() { @Override public String toString() { - return String.format("Partition(topic = %s, partition = %d, leader = %d, replicas = %s, isr = %s", + return String.format("Partition(topic = %s, partition = %d, leader = %s, replicas = %s, isr = %s", topic, partition, - leader.id(), + leader == null ? "none" : leader.id(), fmtNodeIds(replicas), fmtNodeIds(inSyncReplicas)); } diff --git a/clients/src/main/java/org/apache/kafka/common/annotation/InterfaceStability.java b/clients/src/main/java/org/apache/kafka/common/annotation/InterfaceStability.java new file mode 100644 index 0000000000000..0d38f56864587 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/annotation/InterfaceStability.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.common.annotation; + +import java.lang.annotation.Documented; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; + +/** + * Annotation to inform users of how much to rely on a particular package, + * class or method not changing over time. Currently the stability can be + * {@link Stable}, {@link Evolving} or {@link Unstable}.
          + */ +@InterfaceStability.Evolving +public class InterfaceStability { + /** + * Can evolve while retaining compatibility for minor release boundaries.; + * can break compatibility only at major release (ie. at m.0). + */ + @Documented + @Retention(RetentionPolicy.RUNTIME) + public @interface Stable { } + + /** + * Evolving, but can break compatibility at minor release (i.e. m.x) + */ + @Documented + @Retention(RetentionPolicy.RUNTIME) + public @interface Evolving { } + + /** + * No guarantee is provided as to reliability or stability across any + * level of release granularity. + */ + @Documented + @Retention(RetentionPolicy.RUNTIME) + public @interface Unstable { } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/cache/Cache.java b/clients/src/main/java/org/apache/kafka/common/cache/Cache.java new file mode 100644 index 0000000000000..6678e400c48a6 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/cache/Cache.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.cache; + +/** + * Interface for caches, semi-peristent maps which store key-value mappings until either an eviction criteria is met + * or the entries are manually invalidated. Caches are not required to be thread-safe, but some implementations may be. + */ +public interface Cache { + + /** + * Look up a value in the cache. + * @param key the key to + * @return the cached value, or null if it is not present. + */ + V get(K key); + + /** + * Insert an entry into the cache. + * @param key the key to insert + * @param value the value to insert + */ + void put(K key, V value); + + /** + * Manually invalidate a key, clearing its entry from the cache. + * @param key the key to remove + * @return true if the key existed in the cache and the entry was removed or false if it was not present + */ + boolean remove(K key); + + /** + * Get the number of entries in this cache. If this cache is used by multiple threads concurrently, the returned + * value will only be approximate. + * @return the number of entries in the cache + */ + long size(); +} diff --git a/clients/src/main/java/org/apache/kafka/common/cache/LRUCache.java b/clients/src/main/java/org/apache/kafka/common/cache/LRUCache.java new file mode 100644 index 0000000000000..89e6e8747c3b3 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/cache/LRUCache.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.cache; + +import java.util.LinkedHashMap; +import java.util.Map; + +/** + * A cache implementing a least recently used policy. + */ +public class LRUCache implements Cache { + private final LinkedHashMap cache; + + public LRUCache(final int maxSize) { + cache = new LinkedHashMap(16, .75f, true) { + @Override + protected boolean removeEldestEntry(Map.Entry eldest) { + return size() > maxSize; + } + }; + } + + @Override + public V get(K key) { + return cache.get(key); + } + + @Override + public void put(K key, V value) { + cache.put(key, value); + } + + @Override + public boolean remove(K key) { + return cache.remove(key) != null; + } + + @Override + public long size() { + return cache.size(); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/cache/SynchronizedCache.java b/clients/src/main/java/org/apache/kafka/common/cache/SynchronizedCache.java new file mode 100644 index 0000000000000..0e88aa3e12de3 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/cache/SynchronizedCache.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.common.cache; + +/** + * Wrapper for caches that adds simple synchronization to provide a thread-safe cache. Note that this simply adds + * synchronization around each cache method on the underlying unsynchronized cache. It does not add any support for + * atomically checking for existence of an entry and computing and inserting the value if it is missing. + */ +public class SynchronizedCache implements Cache { + private final Cache underlying; + + public SynchronizedCache(Cache underlying) { + this.underlying = underlying; + } + + @Override + public synchronized V get(K key) { + return underlying.get(key); + } + + @Override + public synchronized void put(K key, V value) { + underlying.put(key, value); + } + + @Override + public synchronized boolean remove(K key) { + return underlying.remove(key); + } + + @Override + public synchronized long size() { + return underlying.size(); + } +} 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 8d88610534155..2961e09d7d1f1 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 @@ -3,21 +3,16 @@ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the * License. You may obtain a copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES 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.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; @@ -44,7 +39,7 @@ public class AbstractConfig { private final Map values; @SuppressWarnings("unchecked") - public AbstractConfig(ConfigDef definition, Map originals) { + public AbstractConfig(ConfigDef definition, Map originals, Boolean doLog) { /* check that all the keys are really strings */ for (Object key : originals.keySet()) if (!(key instanceof String)) @@ -52,7 +47,12 @@ public AbstractConfig(ConfigDef definition, Map originals) { this.originals = (Map) originals; this.values = definition.parse(this.originals); this.used = Collections.synchronizedSet(new HashSet()); - logAll(); + if (doLog) + logAll(); + } + + public AbstractConfig(ConfigDef definition, Map originals) { + this(definition, originals, true); } protected Object get(String key) { @@ -62,15 +62,23 @@ protected Object get(String key) { return values.get(key); } - public int getInt(String key) { + public void ignore(String key) { + used.add(key); + } + + public Short getShort(String key) { + return (Short) get(key); + } + + public Integer getInt(String key) { return (Integer) get(key); } - public long getLong(String key) { + public Long getLong(String key) { return (Long) get(key); } - public double getDouble(String key) { + public Double getDouble(String key) { return (Double) get(key); } @@ -97,6 +105,39 @@ public Set unused() { return keys; } + public Properties unusedProperties() { + Set unusedKeys = this.unused(); + Properties unusedProps = new Properties(); + for (String key : unusedKeys) + unusedProps.put(key, this.originals().get(key)); + return unusedProps; + } + + public Map originals() { + Map copy = new HashMap(); + copy.putAll(originals); + return copy; + } + + /** + * Gets all original settings with the given prefix, stripping the prefix before adding it to the output. + * + * @param prefix the prefix to use as a filter + * @return a Map containing the settings with the prefix + */ + public Map originalsWithPrefix(String prefix) { + Map result = new HashMap(); + for (Map.Entry entry : originals.entrySet()) { + if (entry.getKey().startsWith(prefix) && entry.getKey().length() > prefix.length()) + result.put(entry.getKey().substring(prefix.length()), entry.getValue()); + } + return result; + } + + public Map values() { + return new HashMap(values); + } + private void logAll() { StringBuilder b = new StringBuilder(); b.append(getClass().getSimpleName()); @@ -117,13 +158,13 @@ private void logAll() { */ public void logUnused() { for (String key : unused()) - log.warn("The configuration {} = {} was supplied but isn't a known config.", key, this.values.get(key)); + log.warn("The configuration {} = {} was supplied but isn't a known config.", key, this.originals.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. - * + * * @param key The configuration key for the class * @param t The interface the class should implement * @return A configured instance of the class @@ -144,7 +185,12 @@ public List getConfiguredInstances(String key, Class t) { List klasses = getList(key); List objects = new ArrayList(); for (String klass : klasses) { - Class c = getClass(klass); + 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); @@ -157,4 +203,18 @@ public List getConfiguredInstances(String key, Class t) { return objects; } + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + AbstractConfig that = (AbstractConfig) o; + + return originals.equals(that.originals); + } + + @Override + public int hashCode() { + return originals.hashCode(); + } } 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 addc906160883..168990fa8b0ee 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 @@ -3,9 +3,9 @@ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the * License. You may obtain a copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the * specific language governing permissions and limitations under the License. @@ -19,26 +19,29 @@ 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. */ @@ -48,82 +51,136 @@ public class ConfigDef { 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 importance The importance of this config: is this something you will likely need to change. + * + * @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, Importance importance, 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, importance, 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 importance The importance of this config: is this something you will likely need to change. + * + * @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, Importance importance, String documentation) { - return define(name, type, defaultValue, null, importance, 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 importance The importance of this config: is this something you will likely need to change. + * + * @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, Importance importance, String documentation) { - return define(name, type, NO_DEFAULT_VALUE, validator, importance, 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 importance The importance of this config: is this something you will likely need to change. + * + * @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, Importance importance, String documentation) { - return define(name, type, NO_DEFAULT_VALUE, null, importance, 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 - its 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 its default value else value = key.defaultValue; + if (key.validator != null) + key.validator.ensureValid(key.name, value); values.put(key.name, value); } return values; @@ -131,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) { @@ -143,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"); @@ -162,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(); @@ -209,7 +280,7 @@ 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 { @@ -237,6 +308,7 @@ 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) { @@ -268,6 +340,31 @@ else if (max == null) } } + 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, ", ") + "]"; + } + } + private static class ConfigKey { public final String name; public final Type type; @@ -275,8 +372,9 @@ private static class ConfigKey { 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, Importance importance, 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; @@ -286,6 +384,7 @@ public ConfigKey(String name, Type type, Object defaultValue, Validator validato if (this.validator != null) this.validator.ensureValid(name, defaultValue); this.documentation = documentation; + this.required = required; } public boolean hasDefault() { diff --git a/clients/src/main/java/org/apache/kafka/common/config/SSLConfigs.java b/clients/src/main/java/org/apache/kafka/common/config/SSLConfigs.java new file mode 100644 index 0000000000000..dd7b71a46c568 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/config/SSLConfigs.java @@ -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 org.apache.kafka.common.config; + +import javax.net.ssl.TrustManagerFactory; +import javax.net.ssl.KeyManagerFactory; + +public class SSLConfigs { + /* + * 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 PRINCIPAL_BUILDER_CLASS_CONFIG = "principal.builder.class"; + public static final String PRINCIPAL_BUILDER_CLASS_DOC = "principal builder to generate a java Principal. This config is optional for client."; + public static final String DEFAULT_PRINCIPAL_BUILDER_CLASS = "org.apache.kafka.common.security.auth.DefaultPrincipalBuilder"; + + public static final String SSL_PROTOCOL_CONFIG = "ssl.protocol"; + public static final String SSL_PROTOCOL_DOC = "The ssl protocol used to generate SSLContext." + + "Default setting is TLS. Allowed values are SSL, SSLv2, SSLv3, TLS, TLSv1.1, TLSv1.2"; + public static final String DEFAULT_SSL_PROTOCOL = "TLS"; + + public static final String SSL_PROVIDER_CONFIG = "ssl.provider"; + public static final String SSL_PROVIDER_DOC = "The name of the security provider used for SSL connections. Default value is the default security provider of the JVM."; + + public static final String SSL_CIPHER_SUITES_CONFIG = "ssl.cipher.suites"; + public static final String SSL_CIPHER_SUITES_DOC = "A cipher suite is a named combination of authentication, encryption, MAC and key exchange algorithm used to negotiate the security settings for a network connection using TLS or SSL network protocol." + + "By default all the available cipher suites are supported."; + + public static final String SSL_ENABLED_PROTOCOLS_CONFIG = "ssl.enabled.protocols"; + public static final String SSL_ENABLED_PROTOCOLS_DOC = "The list of protocols enabled for SSL connections. " + + "All versions of TLS is enabled by default."; + public static final String DEFAULT_ENABLED_PROTOCOLS = "TLSv1.2,TLSv1.1,TLSv1"; + + public static final String SSL_KEYSTORE_TYPE_CONFIG = "ssl.keystore.type"; + public static final String SSL_KEYSTORE_TYPE_DOC = "The file format of the key store file. " + + "This is optional for client. Default value is JKS"; + public static final String DEFAULT_SSL_KEYSTORE_TYPE = "JKS"; + + public static final String SSL_KEYSTORE_LOCATION_CONFIG = "ssl.keystore.location"; + public static final String SSL_KEYSTORE_LOCATION_DOC = "The location of the key store file. " + + "This is optional for Client and can be used for two-way authentication for client."; + + public static final String SSL_KEYSTORE_PASSWORD_CONFIG = "ssl.keystore.password"; + public static final String SSL_KEYSTORE_PASSWORD_DOC = "The store password for the key store file." + + "This is optional for client and only needed if the ssl.keystore.location configured. "; + + public static final String SSL_KEY_PASSWORD_CONFIG = "ssl.key.password"; + public static final String SSL_KEY_PASSWORD_DOC = "The password of the private key in the key store file. " + + "This is optional for client."; + + public static final String SSL_TRUSTSTORE_TYPE_CONFIG = "ssl.truststore.type"; + public static final String SSL_TRUSTSTORE_TYPE_DOC = "The file format of the trust store file. " + + "Default value is JKS."; + public static final String DEFAULT_SSL_TRUSTSTORE_TYPE = "JKS"; + + public static final String SSL_TRUSTSTORE_LOCATION_CONFIG = "ssl.truststore.location"; + public static final String SSL_TRUSTSTORE_LOCATION_DOC = "The location of the trust store file. "; + public static final String DEFAULT_TRUSTSTORE_LOCATION = "/tmp/ssl.truststore.jks"; + + public static final String SSL_TRUSTSTORE_PASSWORD_CONFIG = "ssl.truststore.password"; + public static final String SSL_TRUSTSTORE_PASSWORD_DOC = "The password for the trust store file. "; + public static final String DEFAULT_TRUSTSTORE_PASSWORD = "truststore_password"; + + public static final String SSL_KEYMANAGER_ALGORITHM_CONFIG = "ssl.keymanager.algorithm"; + public static final String SSL_KEYMANAGER_ALGORITHM_DOC = "The algorithm used by key manager factory for SSL connections. " + + "Default value is the key manager factory algorithm configured for the Java Virtual Machine."; + public static final String DEFAULT_SSL_KEYMANGER_ALGORITHM = KeyManagerFactory.getDefaultAlgorithm(); + + public static final String SSL_TRUSTMANAGER_ALGORITHM_CONFIG = "ssl.trustmanager.algorithm"; + public static final String SSL_TRUSTMANAGER_ALGORITHM_DOC = "The algorithm used by trust manager factory for SSL connections. " + + "Default value is the trust manager factory algorithm configured for the Java Virtual Machine."; + public static final String DEFAULT_SSL_TRUSTMANAGER_ALGORITHM = TrustManagerFactory.getDefaultAlgorithm(); + + public static final String SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG = "ssl.endpoint.identification.algorithm"; + public static final String SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DOC = "The endpoint identification algorithm to validate server hostname using server certificate. "; + + public static final String SSL_CLIENT_AUTH_CONFIG = "ssl.client.auth"; + public static final String SSL_CLIENT_AUTH_DOC = "Configures kafka broker to request client authentication." + + " The following settings are common: " + + "

            " + + "
          • ssl.want.client.auth=required If set to required" + + " client authentication is required." + + "
          • ssl.client.auth=requested This means client authentication is optional." + + " unlike requested , if this option is set client can choose not to provide authentication information about itself" + + "
          • ssl.client.auth=none This means client authentication is not needed."; + + public static final String SSL_NEED_CLIENT_AUTH_DOC = "It can be REQUESTED . " + + "Default value is false"; + public static final Boolean DEFAULT_SSL_NEED_CLIENT_AUTH = false; + +} 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/core/src/main/scala/kafka/network/ByteBufferSend.scala b/clients/src/main/java/org/apache/kafka/common/errors/BrokerNotAvailableException.java similarity index 62% rename from core/src/main/scala/kafka/network/ByteBufferSend.scala rename to clients/src/main/java/org/apache/kafka/common/errors/BrokerNotAvailableException.java index af30042a4c713..f78f0619ed9fc 100644 --- a/core/src/main/scala/kafka/network/ByteBufferSend.scala +++ b/clients/src/main/java/org/apache/kafka/common/errors/BrokerNotAvailableException.java @@ -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,18 @@ * limitations under the License. */ -package kafka.network +package org.apache.kafka.common.errors; + +public class BrokerNotAvailableException extends ApiException { + + private static final long serialVersionUID = 1L; -import java.nio._ -import java.nio.channels._ -import kafka.utils._ + public BrokerNotAvailableException(String message) { + super(message); + } -@nonthreadsafe -private[kafka] class ByteBufferSend(val buffer: ByteBuffer) extends Send { - - var complete: Boolean = false + public BrokerNotAvailableException(String message, Throwable cause) { + super(message, cause); + } - 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 - } - } 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/ControllerMovedException.java b/clients/src/main/java/org/apache/kafka/common/errors/ControllerMovedException.java new file mode 100644 index 0000000000000..8dd7487af2e57 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/ControllerMovedException.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.errors; + +public class ControllerMovedException extends ApiException { + + private static final long serialVersionUID = 1L; + + public ControllerMovedException(String message) { + super(message); + } + + public ControllerMovedException(String message, Throwable cause) { + super(message, cause); + } + +} 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 eaccf276dbfb3..c742580592278 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 @@ -13,7 +13,7 @@ package org.apache.kafka.common.errors; /** - * This exception indicates a record has failed it's internal CRC check, this generally indicates network or disk + * This exception indicates a record has failed its internal CRC check, this generally indicates network or disk * corruption. */ public class CorruptRecordException extends RetriableException { @@ -21,7 +21,7 @@ public class CorruptRecordException extends RetriableException { private static final long serialVersionUID = 1L; public CorruptRecordException() { - super("This message has failed it's CRC checksum or is otherwise corrupt."); + super("This message has failed its CRC checksum or is otherwise corrupt."); } public CorruptRecordException(String message) { 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..fe8ba7a4bd38b --- /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 ApiException { + 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/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/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/NotEnoughReplicasException.java b/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasException.java new file mode 100644 index 0000000000000..1573227acef53 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasException.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; + +/** + * Number of insync replicas for the partition is lower than min.insync.replicas + */ +public class NotEnoughReplicasException extends RetriableException { + private static final long serialVersionUID = 1L; + + public NotEnoughReplicasException() { + super(); + } + + public NotEnoughReplicasException(String message, Throwable cause) { + super(message, cause); + } + + public NotEnoughReplicasException(String message) { + super(message); + } + + public NotEnoughReplicasException(Throwable cause) { + super(cause); + } +} 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/OffsetOutOfRangeException.java b/clients/src/main/java/org/apache/kafka/common/errors/OffsetOutOfRangeException.java index fc7c6e3471b05..4983bc0002ce0 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 @@ -12,6 +12,9 @@ */ package org.apache.kafka.common.errors; +import org.apache.kafka.common.TopicPartition; +import java.util.Map; + /** * This offset is either larger or smaller than the range of offsets the server has for the given partition. * @@ -19,10 +22,15 @@ public class OffsetOutOfRangeException extends RetriableException { private static final long serialVersionUID = 1L; + private Map offsetOutOfRangePartitions = null; public OffsetOutOfRangeException() { } + public OffsetOutOfRangeException(Map offsetOutOfRangePartitions) { + this.offsetOutOfRangePartitions = offsetOutOfRangePartitions; + } + public OffsetOutOfRangeException(String message) { super(message); } @@ -35,4 +43,8 @@ public OffsetOutOfRangeException(String message, Throwable cause) { super(message, cause); } + public Map offsetOutOfRangePartitions() { + return offsetOutOfRangePartitions; + } + } 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/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/UnknownConsumerIdException.java b/clients/src/main/java/org/apache/kafka/common/errors/UnknownConsumerIdException.java new file mode 100644 index 0000000000000..28bfd72fad8c5 --- /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 ApiException { + 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/UnknownTopicOrPartitionException.java b/clients/src/main/java/org/apache/kafka/common/errors/UnknownTopicOrPartitionException.java index ec423bd01298c..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 @@ -15,7 +15,7 @@ /** * This topic/partition doesn't exist */ -public class UnknownTopicOrPartitionException extends RetriableException { +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 3c312011a7ff7..6872049dae7f1 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 @@ -32,6 +32,7 @@ import javax.management.ReflectionException; import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.MetricName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,7 +42,7 @@ public class JmxReporter implements MetricsReporter { private static final Logger log = LoggerFactory.getLogger(JmxReporter.class); - private static final Object lock = new Object(); + private static final Object LOCK = new Object(); private String prefix; private final Map mbeans = new HashMap(); @@ -57,12 +58,11 @@ public JmxReporter(String prefix) { } @Override - public void configure(Map configs) { - } + public void configure(Map configs) {} @Override public void init(List metrics) { - synchronized (lock) { + synchronized (LOCK) { for (KafkaMetric metric : metrics) addAttribute(metric); for (KafkaMbean mbean : mbeans.values()) @@ -72,28 +72,70 @@ public void init(List metrics) { @Override public void metricChange(KafkaMetric metric) { - synchronized (lock) { + synchronized (LOCK) { KafkaMbean mbean = addAttribute(metric); reregister(mbean); } } + @Override + public void metricRemoval(KafkaMetric metric) { + synchronized (LOCK) { + KafkaMbean mbean = removeAttribute(metric); + if (mbean != null) { + if (mbean.metrics.isEmpty()) + unregister(mbean); + else + reregister(mbean); + } + } + } + + private KafkaMbean removeAttribute(KafkaMetric metric) { + MetricName metricName = metric.metricName(); + String mBeanName = getMBeanName(metricName); + KafkaMbean mbean = this.mbeans.get(mBeanName); + if (mbean != null) + mbean.removeAttribute(metricName.name()); + return 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); } } + /** + * @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) { + synchronized (LOCK) { for (KafkaMbean mbean : this.mbeans.values()) unregister(mbean); } @@ -118,29 +160,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() { @@ -172,6 +198,10 @@ public AttributeList getAttributes(String[] names) { } } + public KafkaMetric removeAttribute(String name) { + return this.metrics.remove(name); + } + @Override public MBeanInfo getMBeanInfo() { MBeanAttributeInfo[] attrs = new MBeanAttributeInfo[metrics.size()]; @@ -179,10 +209,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 a7458b50cb16f..e4d3ae834fb02 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,13 +42,8 @@ MetricConfig config() { } @Override - public String name() { - return this.name; - } - - @Override - public String description() { - return this.description; + public MetricName metricName() { + return this.metricName; } @Override @@ -59,6 +53,10 @@ public double value() { } } + public Measurable measurable() { + return this.measurable; + } + double value(long timeMs) { return this.measurable.measure(config, timeMs); } 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 49be4019ac038..be744abe9a165 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 @@ -12,15 +12,23 @@ */ package org.apache.kafka.common.metrics; +import java.io.Closeable; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; +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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A registry of sensors and metrics. @@ -36,20 +44,25 @@ * // 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 ConcurrentMap> childrenSensors; private final List reporters; private final Time time; + private final ScheduledThreadPoolExecutor metricsScheduler; + private static final Logger log = LoggerFactory.getLogger(Metrics.class); /** * Create a metrics repository with no metric reporters and default configuration. @@ -82,12 +95,20 @@ public Metrics(MetricConfig defaultConfig) { */ public Metrics(MetricConfig defaultConfig, List reporters, Time time) { this.config = defaultConfig; - this.sensors = new CopyOnWriteMap(); - this.metrics = new CopyOnWriteMap(); + this.sensors = new CopyOnWriteMap<>(); + this.metrics = new CopyOnWriteMap<>(); + this.childrenSensors = new CopyOnWriteMap<>(); this.reporters = Utils.notNull(reporters); this.time = time; for (MetricsReporter reporter : reporters) reporter.init(new ArrayList()); + this.metricsScheduler = new ScheduledThreadPoolExecutor(1); + // Creating a daemon thread to not block shutdown + this.metricsScheduler.setThreadFactory(new ThreadFactory() { + public Thread newThread(Runnable runnable) { + return Utils.newThread("SensorExpiryThread", runnable, true); + } + }); } /** @@ -128,91 +149,172 @@ public Sensor sensor(String name, Sensor... parents) { * @return The sensor that is created */ public synchronized Sensor sensor(String name, MetricConfig config, Sensor... parents) { + return sensor(name, config, Long.MAX_VALUE, parents); + } + + /** + * 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 inactiveSensorExpirationTimeSeconds If no value if recorded on the Sensor for this duration of time, + * it is eligible for removal + * @param parents The parent sensors + * @return The sensor that is created + */ + public synchronized Sensor sensor(String name, MetricConfig config, long inactiveSensorExpirationTimeSeconds, Sensor... parents) { Sensor s = getSensor(name); if (s == null) { - s = new Sensor(this, name, parents, config == null ? this.config : config, time); + s = new Sensor(this, name, parents, config == null ? this.config : config, time, inactiveSensorExpirationTimeSeconds); this.sensors.put(name, s); + if (parents != null) { + for (Sensor parent : parents) { + List children = childrenSensors.get(parent.name()); + if (children == null) { + children = new ArrayList<>(); + childrenSensors.put(parent, children); + } + children.add(s); + } + } + log.debug("Added sensor with name {}", name); } return s; } /** - * 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 + * Remove a sensor (if it exists), associated metrics and its children. + * + * @param name The name of the sensor to be removed */ - public void addMetric(String name, String description, Measurable measurable) { - addMetric(name, description, null, measurable); + public void removeSensor(String name) { + Sensor sensor = sensors.get(name); + if (sensor != null) { + List childSensors = null; + synchronized (sensor) { + synchronized (this) { + if (sensors.remove(name, sensor)) { + for (KafkaMetric metric : sensor.metrics()) + removeMetric(metric.metricName()); + log.debug("Removed sensor with name {}", name); + childSensors = childrenSensors.remove(sensor); + } + } + } + if (childSensors != null) { + for (Sensor childSensor : childSensors) + removeSensor(childSensor.name()); + } + } } /** * 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); registerMetric(m); } + /** + * Remove a metric if it exists and return it. Return null otherwise. If a metric is removed, `metricRemoval` + * will be invoked for each reporter. + * + * @param metricName The name of the metric + * @return the removed `KafkaMetric` or null if no such metric exists + */ + public synchronized KafkaMetric removeMetric(MetricName metricName) { + KafkaMetric metric = this.metrics.remove(metricName); + if (metric != null) { + for (MetricsReporter reporter : reporters) + reporter.metricRemoval(metric); + } + return metric; + } + /** * Add a MetricReporter */ public synchronized void addReporter(MetricsReporter reporter) { - Utils.notNull(reporter).init(new ArrayList(metrics.values())); + Utils.notNull(reporter).init(new ArrayList<>(metrics.values())); this.reporters.add(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; } + /** + * This iterates over every Sensor and triggers a removeSensor if it has expired + * Package private for testing + */ + class ExpireSensorTask implements Runnable { + public void run() { + for (Map.Entry sensorEntry : sensors.entrySet()) { + // removeSensor also locks the sensor object. This is fine because synchronized is reentrant + // There is however a minor race condition here. Assume we have a parent sensor P and child sensor C. + // Calling record on C would cause a record on P as well. + // So expiration time for P == expiration time for C. If the record on P happens via C just after P is removed, + // that will cause C to also get removed. + // Since the expiration time is typically high it is not expected to be a significant concern + // and thus not necessary to optimize + synchronized (sensorEntry.getValue()) { + if (sensorEntry.getValue().hasExpired()) { + log.debug("Removing expired sensor {}", sensorEntry.getKey()); + removeSensor(sensorEntry.getKey()); + } + } + } + } + } + + /* For testing use only. */ + Map> childrenSensors() { + return Collections.unmodifiableMap(childrenSensors); + } + /** * Close this metrics repository. */ + @Override public void close() { + this.metricsScheduler.shutdown(); + try { + this.metricsScheduler.awaitTermination(30, TimeUnit.SECONDS); + } catch (InterruptedException ex) { + // ignore and continue shutdown + } + 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 7acc19e4e9380..e2a1d80c424df 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 @@ -33,6 +33,12 @@ public interface MetricsReporter extends Configurable { */ public void metricChange(KafkaMetric metric); + /** + * This is called whenever a metric is removed + * @param metric + */ + public void metricRemoval(KafkaMetric metric); + /** * Called when the metrics repository is closed. */ diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Quota.java b/clients/src/main/java/org/apache/kafka/common/metrics/Quota.java index d82bb0c055e63..235b5994cb907 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Quota.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Quota.java @@ -49,4 +49,22 @@ public boolean acceptable(double value) { return (upper && value <= bound) || (!upper && value >= bound); } + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + (int) this.bound; + result = prime * result + (this.upper ? 1 : 0); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (!(obj instanceof Quota)) + return false; + Quota that = (Quota) obj; + return (that.bound == this.bound) && (that.upper == this.upper); + } } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java b/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java index a451e5385c9ec..fbe03f53921ba 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java @@ -28,5 +28,4 @@ public class QuotaViolationException extends KafkaException { public QuotaViolationException(String m) { super(m); } - } 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 25c1faf2887ea..0c5bcb74e6e46 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 @@ -17,7 +17,9 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.concurrent.TimeUnit; +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; @@ -36,16 +38,20 @@ public final class Sensor { private final List metrics; private final MetricConfig config; private final Time time; + private volatile long lastRecordTime; + private final long inactiveSensorExpirationTimeMs; - Sensor(Metrics registry, String name, Sensor[] parents, MetricConfig config, Time time) { + Sensor(Metrics registry, String name, Sensor[] parents, MetricConfig config, Time time, long inactiveSensorExpirationTimeSeconds) { super(); this.registry = registry; this.name = Utils.notNull(name); this.parents = parents == null ? new Sensor[0] : parents; - this.metrics = new ArrayList(); - this.stats = new ArrayList(); + this.metrics = new ArrayList<>(); + this.stats = new ArrayList<>(); this.config = config; this.time = time; + this.inactiveSensorExpirationTimeMs = TimeUnit.MILLISECONDS.convert(inactiveSensorExpirationTimeSeconds, TimeUnit.SECONDS); + this.lastRecordTime = time.milliseconds(); checkForest(new HashSet()); } @@ -90,6 +96,7 @@ public void record(double value) { * bound */ public void record(double value, long timeMs) { + this.lastRecordTime = time.milliseconds(); synchronized (this) { // increment all the stats for (int i = 0; i < this.stats.size(); i++) @@ -111,8 +118,14 @@ private void checkQuotas(long timeMs) { if (config != null) { Quota quota = config.quota(); if (quota != null) { - if (!quota.acceptable(metric.value(timeMs))) - throw new QuotaViolationException("Metric " + metric.name() + " is in violation of its quota of " + quota.bound()); + double value = metric.value(timeMs); + if (!quota.acceptable(value)) { + throw new QuotaViolationException(String.format( + "(%s) violated quota. Actual: (%f), Threshold: (%f)", + metric.metricName(), + quota.bound(), + value)); + } } } } @@ -134,52 +147,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); @@ -188,8 +179,15 @@ public synchronized void add(String name, String description, MeasurableStat sta this.stats.add(stat); } + /** + * Return true if the Sensor is eligible for removal due to inactivity. + * false otherwise + */ + public boolean hasExpired() { + return (time.milliseconds() - this.lastRecordTime) > this.inactiveSensorExpirationTimeMs; + } + synchronized List metrics() { return Collections.unmodifiableList(this.metrics); } - } 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/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 c70d577ada8c0..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 @@ -59,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); } 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 a5838b3894906..9dfc457b0912e 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 @@ -18,6 +18,7 @@ import org.apache.kafka.common.metrics.MeasurableStat; import org.apache.kafka.common.metrics.MetricConfig; + /** * 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, @@ -58,26 +59,52 @@ public void record(MetricConfig config, double value, long timeMs) { @Override public double measure(MetricConfig config, long now) { double value = stat.measure(config, now); - double elapsed = convert(now - stat.oldest(now).lastWindowMs); - return value / elapsed; + return value / convert(windowSize(config, now)); + } + + public long windowSize(MetricConfig config, long now) { + // purge old samples before we compute the window size + stat.purgeObsoleteSamples(config, now); + + /* + * Here we check the total amount of time elapsed since the oldest non-obsolete window. + * This give the total windowSize of the batch which is the time used for Rate computation. + * However, there is an issue if we do not have sufficient data for e.g. if only 1 second has elapsed in a 30 second + * window, the measured rate will be very high. + * Hence we assume that the elapsed time is always N-1 complete windows plus whatever fraction of the final window is complete. + * + * Note that we could simply count the amount of time elapsed in the current window and add n-1 windows to get the total time, + * but this approach does not account for sleeps. SampledStat only creates samples whenever record is called, + * if no record is called for a period of time that time is not accounted for in windowSize and produces incorrect results. + */ + long totalElapsedTimeMs = now - stat.oldest(now).lastWindowMs; + // Check how many full windows of data we have currently retained + int numFullWindows = (int) (totalElapsedTimeMs / config.timeWindowMs()); + int minFullWindows = config.samples() - 1; + + // If the available windows are less than the minimum required, add the difference to the totalElapsedTime + if (numFullWindows < minFullWindows) + totalElapsedTimeMs += (minFullWindows - numFullWindows) * config.timeWindowMs(); + + return totalElapsedTimeMs; } - private double convert(long time) { + private double convert(long timeMs) { switch (unit) { case NANOSECONDS: - return time * 1000.0 * 1000.0; + return timeMs * 1000.0 * 1000.0; case MICROSECONDS: - return time * 1000.0; + return timeMs * 1000.0; case MILLISECONDS: - return time; + return timeMs; case SECONDS: - return time / (1000.0); + return timeMs / 1000.0; case MINUTES: - return time / (60.0 * 1000.0); + return timeMs / (60.0 * 1000.0); case HOURS: - return time / (60.0 * 60.0 * 1000.0); + return timeMs / (60.0 * 60.0 * 1000.0); case DAYS: - return time / (24.0 * 60.0 * 60.0 * 1000.0); + return timeMs / (24.0 * 60.0 * 60.0 * 1000.0); default: throw new IllegalStateException("Unknown unit: " + unit); } diff --git a/clients/src/main/java/org/apache/kafka/common/network/Authenticator.java b/clients/src/main/java/org/apache/kafka/common/network/Authenticator.java new file mode 100644 index 0000000000000..261f571715f7d --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/network/Authenticator.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.network; + +/** + * Authentication for Channel + */ + +import java.io.IOException; +import java.security.Principal; + +import org.apache.kafka.common.security.auth.PrincipalBuilder; +import org.apache.kafka.common.KafkaException; + +public interface Authenticator { + + /** + * configures Authenticator using principalbuilder and transportLayer. + * @param TransportLayer transportLayer + * @param PrincipalBuilder principalBuilder + */ + void configure(TransportLayer transportLayer, PrincipalBuilder principalBuilder); + + /** + * Implements any authentication mechanism. Use transportLayer to read or write tokens. + * If no further authentication needs to be done returns. + */ + void authenticate() throws IOException; + + /** + * Returns Principal using PrincipalBuilder + */ + Principal principal() throws KafkaException; + + /** + * returns true if authentication is complete otherwise returns false; + */ + boolean complete(); + + /** + * Closes this Authenticator + * + * @throws IOException if any I/O error occurs + */ + void close() throws IOException; + +} 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 c8213e156ec9c..d7357b2ce61fb 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 @@ -3,9 +3,9 @@ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the * License. You may obtain a copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the * specific language governing permissions and limitations under the License. @@ -22,12 +22,13 @@ */ public class ByteBufferSend implements Send { - private final int destination; + private final String destination; protected final ByteBuffer[] buffers; private int remaining; private int size; + private boolean pending = false; - public ByteBufferSend(int destination, ByteBuffer... buffers) { + public ByteBufferSend(String destination, ByteBuffer... buffers) { super(); this.destination = destination; this.buffers = buffers; @@ -37,26 +38,17 @@ public ByteBufferSend(int destination, ByteBuffer... buffers) { } @Override - public int destination() { + public String destination() { return destination; } @Override public boolean completed() { - return remaining <= 0; + return remaining <= 0 && !pending; } @Override - public ByteBuffer[] reify() { - return this.buffers; - } - - @Override - public int remaining() { - return this.remaining; - } - - public int size() { + public long size() { return this.size; } @@ -64,9 +56,14 @@ public int size() { 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; + // This is temporary workaround. As Send , Receive interfaces are being used by BlockingChannel. + // Once BlockingChannel is removed we can make Send, Receive to work with transportLayer rather than + // GatheringByteChannel or ScatteringByteChannel. + if (channel instanceof TransportLayer) + pending = ((TransportLayer) channel).hasPendingWrites(); + return written; } - } diff --git a/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilder.java b/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilder.java new file mode 100644 index 0000000000000..52a7aab1e15ea --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilder.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.network; + +import java.util.Map; +import java.nio.channels.SelectionKey; + +import org.apache.kafka.common.KafkaException; + +/** + * A ChannelBuilder interface to build Channel based on configs + */ +public interface ChannelBuilder { + + /** + * Configure this class with the given key-value pairs + */ + void configure(Map configs) throws KafkaException; + + + /** + * returns a Channel with TransportLayer and Authenticator configured. + * @param id channel id + * @param key SelectionKey + */ + KafkaChannel buildChannel(String id, SelectionKey key, int maxReceiveSize) throws KafkaException; + + + /** + * Closes ChannelBuilder + */ + void close(); + +} diff --git a/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java b/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java new file mode 100644 index 0000000000000..2332d3f42815e --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.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.common.network; + +import org.apache.kafka.common.protocol.SecurityProtocol; +import org.apache.kafka.common.security.ssl.SSLFactory; + +import java.util.Map; + +public class ChannelBuilders { + + private ChannelBuilders() { } + + /** + * @param securityProtocol the securityProtocol + * @param mode the SSL mode, it must be non-null if `securityProcol` is `SSL` and it is ignored otherwise + * @param configs client/server configs + * @return the configured `ChannelBuilder` + * @throws IllegalArgumentException if `mode` invariants described above is not maintained + */ + public static ChannelBuilder create(SecurityProtocol securityProtocol, SSLFactory.Mode mode, Map configs) { + ChannelBuilder channelBuilder = null; + + switch (securityProtocol) { + case SSL: + if (mode == null) + throw new IllegalArgumentException("`mode` must be non-null if `securityProtocol` is `SSL`"); + channelBuilder = new SSLChannelBuilder(mode); + break; + case PLAINTEXT: + case TRACE: + channelBuilder = new PlaintextChannelBuilder(); + break; + default: + throw new IllegalArgumentException("Unexpected securityProtocol " + securityProtocol); + } + + channelBuilder.configure(configs); + return channelBuilder; + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java new file mode 100644 index 0000000000000..813a4aafeb927 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.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.network; + +import java.security.Principal; +import java.io.IOException; + +import org.apache.kafka.common.security.auth.PrincipalBuilder; +import org.apache.kafka.common.KafkaException; + +public class DefaultAuthenticator implements Authenticator { + + private TransportLayer transportLayer; + private PrincipalBuilder principalBuilder; + private Principal principal; + + public void configure(TransportLayer transportLayer, PrincipalBuilder principalBuilder) { + this.transportLayer = transportLayer; + this.principalBuilder = principalBuilder; + } + + /** + * No-Op for default authenticator + */ + public void authenticate() throws IOException {} + + /** + * Constructs Principal using configured principalBuilder. + * @return Principal + * @throws KafkaException + */ + public Principal principal() throws KafkaException { + if (principal == null) + principal = principalBuilder.buildPrincipal(transportLayer, this); + return principal; + } + + public void close() throws IOException {} + + /** + * DefaultAuthenticator doesn't implement any additional authentication mechanism. + * @returns true + */ + public boolean complete() { + return true; + } + +} 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/KafkaChannel.java b/clients/src/main/java/org/apache/kafka/common/network/KafkaChannel.java new file mode 100644 index 0000000000000..172f4cda38d2f --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/network/KafkaChannel.java @@ -0,0 +1,174 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.net.InetAddress; +import java.net.Socket; +import java.nio.channels.SelectionKey; + +import java.security.Principal; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class KafkaChannel { + private static final Logger log = LoggerFactory.getLogger(KafkaChannel.class); + private final String id; + private TransportLayer transportLayer; + private Authenticator authenticator; + private NetworkReceive receive; + private Send send; + private int maxReceiveSize; + + public KafkaChannel(String id, TransportLayer transportLayer, Authenticator authenticator, int maxReceiveSize) throws IOException { + this.id = id; + this.transportLayer = transportLayer; + this.authenticator = authenticator; + this.maxReceiveSize = maxReceiveSize; + } + + public void close() throws IOException { + transportLayer.close(); + authenticator.close(); + } + + /** + * returns user principal for the session + * In case of PLAINTEXT and No Authentication returns ANONYMOUS as the userPrincipal + * If SSL used without any SASL Authentication returns SSLSession.peerPrincipal + */ + public Principal principal() throws IOException { + return authenticator.principal(); + } + + /** + * Does handshake of transportLayer and Authentication using configured authenticator + */ + public void prepare() throws IOException { + if (transportLayer.ready() && authenticator.complete()) + return; + if (!transportLayer.ready()) + transportLayer.handshake(); + if (transportLayer.ready() && !authenticator.complete()) + authenticator.authenticate(); + } + + public void disconnect() { + transportLayer.disconnect(); + } + + + public void finishConnect() throws IOException { + transportLayer.finishConnect(); + } + + public boolean isConnected() { + return transportLayer.isConnected(); + } + + public String id() { + return id; + } + + public void mute() { + transportLayer.removeInterestOps(SelectionKey.OP_READ); + } + + public void unmute() { + transportLayer.addInterestOps(SelectionKey.OP_READ); + } + + public boolean isMute() { + return transportLayer.isMute(); + } + + public boolean ready() { + return transportLayer.ready() && authenticator.complete(); + } + + public boolean hasSend() { + return send != null; + } + + /** + * Returns the address to which this channel's socket is connected or `null` if the socket has never been connected. + * + * If the socket was connected prior to being closed, then this method will continue to return the + * connected address after the socket is closed. + */ + public InetAddress socketAddress() { + return transportLayer.socketChannel().socket().getInetAddress(); + } + + public String socketDescription() { + Socket socket = transportLayer.socketChannel().socket(); + if (socket.getInetAddress() == null) + return socket.getLocalAddress().toString(); + return socket.getInetAddress().toString(); + } + + public void setSend(Send send) { + if (this.send != null) + throw new IllegalStateException("Attempt to begin a send operation with prior send operation still in progress."); + this.send = send; + this.transportLayer.addInterestOps(SelectionKey.OP_WRITE); + } + + public NetworkReceive read() throws IOException { + NetworkReceive result = null; + + if (receive == null) { + receive = new NetworkReceive(maxReceiveSize, id); + } + + receive(receive); + if (receive.complete()) { + receive.payload().rewind(); + result = receive; + receive = null; + } + return result; + } + + public Send write() throws IOException { + Send result = null; + if (send != null && send(send)) { + result = send; + send = null; + } + return result; + } + + private long receive(NetworkReceive receive) throws IOException { + long result = receive.readFrom(transportLayer); + return result; + } + + private boolean send(Send send) throws IOException { + send.writeTo(transportLayer); + if (send.completed()) + transportLayer.removeInterestOps(SelectionKey.OP_WRITE); + + return send.completed(); + } + +} 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 dcc639a4bb451..409775cd9785c 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 @@ -3,9 +3,9 @@ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the * License. You may obtain a copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the * specific language governing permissions and limitations under the License. @@ -15,6 +15,7 @@ import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.channels.ReadableByteChannel; import java.nio.channels.ScatteringByteChannel; /** @@ -22,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; } @@ -48,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); @@ -63,10 +84,13 @@ 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) { 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/PlaintextChannelBuilder.java b/clients/src/main/java/org/apache/kafka/common/network/PlaintextChannelBuilder.java new file mode 100644 index 0000000000000..76dbf93a02a81 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/network/PlaintextChannelBuilder.java @@ -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 org.apache.kafka.common.network; + +import java.nio.channels.SelectionKey; +import java.util.Map; + +import org.apache.kafka.common.security.auth.PrincipalBuilder; +import org.apache.kafka.common.config.SSLConfigs; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.common.KafkaException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class PlaintextChannelBuilder implements ChannelBuilder { + private static final Logger log = LoggerFactory.getLogger(PlaintextChannelBuilder.class); + private PrincipalBuilder principalBuilder; + + public void configure(Map configs) throws KafkaException { + try { + this.principalBuilder = (PrincipalBuilder) Utils.newInstance((Class) configs.get(SSLConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG)); + this.principalBuilder.configure(configs); + } catch (Exception e) { + throw new KafkaException(e); + } + } + + public KafkaChannel buildChannel(String id, SelectionKey key, int maxReceiveSize) throws KafkaException { + KafkaChannel channel = null; + try { + PlaintextTransportLayer transportLayer = new PlaintextTransportLayer(key); + Authenticator authenticator = new DefaultAuthenticator(); + authenticator.configure(transportLayer, this.principalBuilder); + channel = new KafkaChannel(id, transportLayer, authenticator, maxReceiveSize); + } catch (Exception e) { + log.warn("Failed to create channel due to ", e); + throw new KafkaException(e); + } + return channel; + } + + public void close() { + this.principalBuilder.close(); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/network/PlaintextTransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/PlaintextTransportLayer.java new file mode 100644 index 0000000000000..1149c99d7ea50 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/network/PlaintextTransportLayer.java @@ -0,0 +1,222 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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; + +/* + * Transport layer for PLAINTEXT communication + */ + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.channels.SocketChannel; +import java.nio.channels.SelectionKey; + +import java.security.Principal; + +import org.apache.kafka.common.security.auth.KafkaPrincipal; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class PlaintextTransportLayer implements TransportLayer { + private static final Logger log = LoggerFactory.getLogger(PlaintextTransportLayer.class); + private final SelectionKey key; + private final SocketChannel socketChannel; + private final Principal principal = KafkaPrincipal.ANONYMOUS; + + public PlaintextTransportLayer(SelectionKey key) throws IOException { + this.key = key; + this.socketChannel = (SocketChannel) key.channel(); + } + + @Override + public boolean ready() { + return true; + } + + @Override + public void finishConnect() throws IOException { + socketChannel.finishConnect(); + int ops = key.interestOps(); + ops &= ~SelectionKey.OP_CONNECT; + ops |= SelectionKey.OP_READ; + key.interestOps(ops); + } + + @Override + public void disconnect() { + key.cancel(); + } + + @Override + public SocketChannel socketChannel() { + return socketChannel; + } + + @Override + public boolean isOpen() { + return socketChannel.isOpen(); + } + + @Override + public boolean isConnected() { + return socketChannel.isConnected(); + } + + /** + * Closes this channel + * + * @throws IOException If I/O error occurs + */ + @Override + public void close() throws IOException { + socketChannel.socket().close(); + socketChannel.close(); + key.attach(null); + key.cancel(); + } + + /** + * Performs SSL handshake hence is a no-op for the non-secure + * implementation + * @throws IOException + */ + @Override + public void handshake() throws IOException {} + + /** + * Reads a sequence of bytes from this channel into the given buffer. + * + * @param dst The buffer into which bytes are to be transferred + * @return The number of bytes read, possible zero or -1 if the channel has reached end-of-stream + * @throws IOException if some other I/O error occurs + */ + @Override + public int read(ByteBuffer dst) throws IOException { + return socketChannel.read(dst); + } + + /** + * Reads a sequence of bytes from this channel into the given buffers. + * + * @param dsts - The buffers into which bytes are to be transferred. + * @return The number of bytes read, possibly zero, or -1 if the channel has reached end-of-stream. + * @throws IOException if some other I/O error occurs + */ + @Override + public long read(ByteBuffer[] dsts) throws IOException { + return socketChannel.read(dsts); + } + + /** + * Reads a sequence of bytes from this channel into a subsequence of the given buffers. + * @param dsts - The buffers into which bytes are to be transferred + * @param offset - The offset within the buffer array of the first buffer into which bytes are to be transferred; must be non-negative and no larger than dsts.length. + * @param length - The maximum number of buffers to be accessed; must be non-negative and no larger than dsts.length - offset + * @returns The number of bytes read, possibly zero, or -1 if the channel has reached end-of-stream. + * @throws IOException if some other I/O error occurs + */ + @Override + public long read(ByteBuffer[] dsts, int offset, int length) throws IOException { + return socketChannel.read(dsts, offset, length); + } + + /** + * Writes a sequence of bytes to this channel from the given buffer. + * + * @param src The buffer from which bytes are to be retrieved + * @returns The number of bytes read, possibly zero, or -1 if the channel has reached end-of-stream + * @throws IOException If some other I/O error occurs + */ + @Override + public int write(ByteBuffer src) throws IOException { + return socketChannel.write(src); + } + + /** + * Writes a sequence of bytes to this channel from the given buffer. + * + * @param srcs The buffer from which bytes are to be retrieved + * @returns The number of bytes read, possibly zero, or -1 if the channel has reached end-of-stream + * @throws IOException If some other I/O error occurs + */ + @Override + public long write(ByteBuffer[] srcs) throws IOException { + return socketChannel.write(srcs); + } + + /** + * Writes a sequence of bytes to this channel from the subsequence of the given buffers. + * + * @param srcs The buffers from which bytes are to be retrieved + * @param offset The offset within the buffer array of the first buffer from which bytes are to be retrieved; must be non-negative and no larger than srcs.length. + * @param length - The maximum number of buffers to be accessed; must be non-negative and no larger than srcs.length - offset. + * @return returns no.of bytes written , possibly zero. + * @throws IOException If some other I/O error occurs + */ + @Override + public long write(ByteBuffer[] srcs, int offset, int length) throws IOException { + return socketChannel.write(srcs, offset, length); + } + + /** + * always returns false as there will be not be any + * pending writes since we directly write to socketChannel. + */ + @Override + public boolean hasPendingWrites() { + return false; + } + + /** + * Returns ANONYMOUS as Principal. + */ + @Override + public Principal peerPrincipal() throws IOException { + return principal; + } + + /** + * Adds the interestOps to selectionKey. + * @param ops + */ + @Override + public void addInterestOps(int ops) { + key.interestOps(key.interestOps() | ops); + + } + + /** + * Removes the interestOps from selectionKey. + * @param ops + */ + @Override + public void removeInterestOps(int ops) { + key.interestOps(key.interestOps() & ~ops); + } + + @Override + public boolean isMute() { + return key.isValid() && (key.interestOps() & SelectionKey.OP_READ) == 0; + } + + @Override + public long transferFrom(FileChannel fileChannel, long position, long count) throws IOException { + return fileChannel.transferTo(position, count, socketChannel); + } +} 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/SSLChannelBuilder.java b/clients/src/main/java/org/apache/kafka/common/network/SSLChannelBuilder.java new file mode 100644 index 0000000000000..e2cce5cac1d54 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/network/SSLChannelBuilder.java @@ -0,0 +1,75 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES 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.channels.SelectionKey; +import java.nio.channels.SocketChannel; +import java.util.Map; + +import org.apache.kafka.common.security.auth.PrincipalBuilder; +import org.apache.kafka.common.security.ssl.SSLFactory; +import org.apache.kafka.common.config.SSLConfigs; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.common.KafkaException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SSLChannelBuilder implements ChannelBuilder { + private static final Logger log = LoggerFactory.getLogger(SSLChannelBuilder.class); + private SSLFactory sslFactory; + private PrincipalBuilder principalBuilder; + private SSLFactory.Mode mode; + + public SSLChannelBuilder(SSLFactory.Mode mode) { + this.mode = mode; + } + + public void configure(Map configs) throws KafkaException { + try { + this.sslFactory = new SSLFactory(mode); + this.sslFactory.configure(configs); + this.principalBuilder = (PrincipalBuilder) Utils.newInstance((Class) configs.get(SSLConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG)); + this.principalBuilder.configure(configs); + } catch (Exception e) { + throw new KafkaException(e); + } + } + + public KafkaChannel buildChannel(String id, SelectionKey key, int maxReceiveSize) throws KafkaException { + KafkaChannel channel = null; + try { + SSLTransportLayer transportLayer = buildTransportLayer(sslFactory, id, key); + Authenticator authenticator = new DefaultAuthenticator(); + authenticator.configure(transportLayer, this.principalBuilder); + channel = new KafkaChannel(id, transportLayer, authenticator, maxReceiveSize); + } catch (Exception e) { + log.info("Failed to create channel due to ", e); + throw new KafkaException(e); + } + return channel; + } + + public void close() { + this.principalBuilder.close(); + } + + protected SSLTransportLayer buildTransportLayer(SSLFactory sslFactory, String id, SelectionKey key) throws IOException { + SocketChannel socketChannel = (SocketChannel) key.channel(); + SSLTransportLayer transportLayer = new SSLTransportLayer(id, key, + sslFactory.createSSLEngine(socketChannel.socket().getInetAddress().getHostName(), + socketChannel.socket().getPort())); + transportLayer.startHandshake(); + return transportLayer; + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java new file mode 100644 index 0000000000000..35ea9aae56abb --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java @@ -0,0 +1,716 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.io.EOFException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.channels.SocketChannel; +import java.nio.channels.SelectionKey; +import java.nio.channels.CancelledKeyException; + +import java.security.Principal; +import javax.net.ssl.SSLEngine; +import javax.net.ssl.SSLEngineResult; +import javax.net.ssl.SSLEngineResult.HandshakeStatus; +import javax.net.ssl.SSLEngineResult.Status; +import javax.net.ssl.SSLException; +import javax.net.ssl.SSLSession; +import javax.net.ssl.SSLPeerUnverifiedException; + +import org.apache.kafka.common.security.auth.KafkaPrincipal; +import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/* + * Transport layer for SSL communication + */ + +public class SSLTransportLayer implements TransportLayer { + private static final Logger log = LoggerFactory.getLogger(SSLTransportLayer.class); + private final String channelId; + private final SSLEngine sslEngine; + private final SelectionKey key; + private final SocketChannel socketChannel; + private HandshakeStatus handshakeStatus; + private SSLEngineResult handshakeResult; + private boolean handshakeComplete = false; + private boolean closing = false; + private ByteBuffer netReadBuffer; + private ByteBuffer netWriteBuffer; + private ByteBuffer appReadBuffer; + private ByteBuffer emptyBuf = ByteBuffer.allocate(0); + + public SSLTransportLayer(String channelId, SelectionKey key, SSLEngine sslEngine) throws IOException { + this.channelId = channelId; + this.key = key; + this.socketChannel = (SocketChannel) key.channel(); + this.sslEngine = sslEngine; + } + + /** + * starts sslEngine handshake process + */ + protected void startHandshake() throws IOException { + + this.netReadBuffer = ByteBuffer.allocate(netReadBufferSize()); + this.netWriteBuffer = ByteBuffer.allocate(netWriteBufferSize()); + this.appReadBuffer = ByteBuffer.allocate(applicationBufferSize()); + + //clear & set netRead & netWrite buffers + netWriteBuffer.position(0); + netWriteBuffer.limit(0); + netReadBuffer.position(0); + netReadBuffer.limit(0); + handshakeComplete = false; + closing = false; + //initiate handshake + sslEngine.beginHandshake(); + handshakeStatus = sslEngine.getHandshakeStatus(); + } + + @Override + public boolean ready() { + return handshakeComplete; + } + + /** + * does socketChannel.finishConnect() + */ + @Override + public void finishConnect() throws IOException { + socketChannel.finishConnect(); + key.interestOps(key.interestOps() & ~SelectionKey.OP_CONNECT | SelectionKey.OP_READ); + } + + /** + * disconnects selectionKey. + */ + @Override + public void disconnect() { + key.cancel(); + } + + @Override + public SocketChannel socketChannel() { + return socketChannel; + } + + @Override + public boolean isOpen() { + return socketChannel.isOpen(); + } + + @Override + public boolean isConnected() { + return socketChannel.isConnected(); + } + + + /** + * Sends a SSL close message and closes socketChannel. + * @throws IOException if an I/O error occurs + * @throws IOException if there is data on the outgoing network buffer and we are unable to flush it + */ + @Override + public void close() throws IOException { + if (closing) return; + closing = true; + sslEngine.closeOutbound(); + try { + if (!flush(netWriteBuffer)) { + throw new IOException("Remaining data in the network buffer, can't send SSL close message."); + } + //prep the buffer for the close message + netWriteBuffer.clear(); + //perform the close, since we called sslEngine.closeOutbound + SSLEngineResult handshake = sslEngine.wrap(emptyBuf, netWriteBuffer); + //we should be in a close state + if (handshake.getStatus() != SSLEngineResult.Status.CLOSED) { + throw new IOException("Invalid close state, will not send network data."); + } + netWriteBuffer.flip(); + flush(netWriteBuffer); + socketChannel.socket().close(); + socketChannel.close(); + } catch (IOException ie) { + log.warn("Failed to send SSL Close message ", ie); + } + key.attach(null); + key.cancel(); + } + + /** + * returns true if there are any pending contents in netWriteBuffer + */ + @Override + public boolean hasPendingWrites() { + return netWriteBuffer.hasRemaining(); + } + + /** + * Flushes the buffer to the network, non blocking + * @param buf ByteBuffer + * @return boolean true if the buffer has been emptied out, false otherwise + * @throws IOException + */ + private boolean flush(ByteBuffer buf) throws IOException { + int remaining = buf.remaining(); + if (remaining > 0) { + int written = socketChannel.write(buf); + return written >= remaining; + } + return true; + } + + /** + * Performs SSL handshake, non blocking. + * Before application data (kafka protocols) can be sent client & kafka broker must + * perform ssl handshake. + * During the handshake SSLEngine generates encrypted data that will be transported over socketChannel. + * Each SSLEngine operation generates SSLEngineResult , of which SSLEngineResult.handshakeStatus field is used to + * determine what operation needs to occur to move handshake along. + * A typical handshake might look like this. + * +-------------+----------------------------------+-------------+ + * | client | SSL/TLS message | HSStatus | + * +-------------+----------------------------------+-------------+ + * | wrap() | ClientHello | NEED_UNWRAP | + * | unwrap() | ServerHello/Cert/ServerHelloDone | NEED_WRAP | + * | wrap() | ClientKeyExchange | NEED_WRAP | + * | wrap() | ChangeCipherSpec | NEED_WRAP | + * | wrap() | Finished | NEED_UNWRAP | + * | unwrap() | ChangeCipherSpec | NEED_UNWRAP | + * | unwrap() | Finished | FINISHED | + * +-------------+----------------------------------+-------------+ + * + * @throws IOException + */ + @Override + public void handshake() throws IOException { + boolean read = key.isReadable(); + boolean write = key.isWritable(); + handshakeComplete = false; + handshakeStatus = sslEngine.getHandshakeStatus(); + if (!flush(netWriteBuffer)) { + key.interestOps(key.interestOps() | SelectionKey.OP_WRITE); + return; + } + try { + switch (handshakeStatus) { + case NEED_TASK: + log.trace("SSLHandshake NEED_TASK channelId {}, appReadBuffer pos {}, netReadBuffer pos {}, netWriteBuffer pos {}", + channelId, appReadBuffer.position(), netReadBuffer.position(), netWriteBuffer.position()); + handshakeStatus = runDelegatedTasks(); + break; + case NEED_WRAP: + log.trace("SSLHandshake NEED_WRAP channelId {}, appReadBuffer pos {}, netReadBuffer pos {}, netWriteBuffer pos {}", + channelId, appReadBuffer.position(), netReadBuffer.position(), netWriteBuffer.position()); + handshakeResult = handshakeWrap(write); + if (handshakeResult.getStatus() == Status.BUFFER_OVERFLOW) { + int currentNetWriteBufferSize = netWriteBufferSize(); + netWriteBuffer.compact(); + netWriteBuffer = Utils.ensureCapacity(netWriteBuffer, currentNetWriteBufferSize); + netWriteBuffer.flip(); + if (netWriteBuffer.limit() >= currentNetWriteBufferSize) { + throw new IllegalStateException("Buffer overflow when available data size (" + netWriteBuffer.limit() + + ") >= network buffer size (" + currentNetWriteBufferSize + ")"); + } + } else if (handshakeResult.getStatus() == Status.BUFFER_UNDERFLOW) { + throw new IllegalStateException("Should not have received BUFFER_UNDERFLOW during handshake WRAP."); + } else if (handshakeResult.getStatus() == Status.CLOSED) { + throw new EOFException(); + } + log.trace("SSLHandshake NEED_WRAP channelId {}, handshakeResult {}, appReadBuffer pos {}, netReadBuffer pos {}, netWriteBuffer pos {}", + channelId, handshakeResult, appReadBuffer.position(), netReadBuffer.position(), netWriteBuffer.position()); + //if handshake status is not NEED_UNWRAP or unable to flush netWriteBuffer contents + //we will break here otherwise we can do need_unwrap in the same call. + if (handshakeStatus != HandshakeStatus.NEED_UNWRAP || !flush(netWriteBuffer)) { + key.interestOps(key.interestOps() | SelectionKey.OP_WRITE); + break; + } + case NEED_UNWRAP: + log.trace("SSLHandshake NEED_UNWRAP channelId {}, appReadBuffer pos {}, netReadBuffer pos {}, netWriteBuffer pos {}", + channelId, appReadBuffer.position(), netReadBuffer.position(), netWriteBuffer.position()); + do { + handshakeResult = handshakeUnwrap(read); + if (handshakeResult.getStatus() == Status.BUFFER_OVERFLOW) { + int currentAppBufferSize = applicationBufferSize(); + appReadBuffer = Utils.ensureCapacity(appReadBuffer, currentAppBufferSize); + if (appReadBuffer.position() > currentAppBufferSize) { + throw new IllegalStateException("Buffer underflow when available data size (" + appReadBuffer.position() + + ") > packet buffer size (" + currentAppBufferSize + ")"); + } + } + } while (handshakeResult.getStatus() == Status.BUFFER_OVERFLOW); + if (handshakeResult.getStatus() == Status.BUFFER_UNDERFLOW) { + int currentNetReadBufferSize = netReadBufferSize(); + netReadBuffer = Utils.ensureCapacity(netReadBuffer, currentNetReadBufferSize); + if (netReadBuffer.position() >= currentNetReadBufferSize) { + throw new IllegalStateException("Buffer underflow when there is available data"); + } + } else if (handshakeResult.getStatus() == Status.CLOSED) { + throw new EOFException("SSL handshake status CLOSED during handshake UNWRAP"); + } + log.trace("SSLHandshake NEED_UNWRAP channelId {}, handshakeResult {}, appReadBuffer pos {}, netReadBuffer pos {}, netWriteBuffer pos {}", + channelId, handshakeResult, appReadBuffer.position(), netReadBuffer.position(), netWriteBuffer.position()); + + //if handshakeStatus completed than fall-through to finished status. + //after handshake is finished there is no data left to read/write in socketChannel. + //so the selector won't invoke this channel if we don't go through the handshakeFinished here. + if (handshakeStatus != HandshakeStatus.FINISHED) { + if (handshakeStatus == HandshakeStatus.NEED_WRAP) { + key.interestOps(key.interestOps() | SelectionKey.OP_WRITE); + } else if (handshakeStatus == HandshakeStatus.NEED_UNWRAP) { + key.interestOps(key.interestOps() & ~SelectionKey.OP_WRITE); + } + break; + } + case FINISHED: + handshakeFinished(); + break; + case NOT_HANDSHAKING: + handshakeFinished(); + break; + default: + throw new IllegalStateException(String.format("Unexpected status [%s]", handshakeStatus)); + } + + } catch (SSLException e) { + handshakeFailure(); + throw e; + } + } + + + /** + * Executes the SSLEngine tasks needed. + * @return HandshakeStatus + */ + private HandshakeStatus runDelegatedTasks() { + for (;;) { + Runnable task = delegatedTask(); + if (task == null) { + break; + } + task.run(); + } + return sslEngine.getHandshakeStatus(); + } + + /** + * Checks if the handshake status is finished + * Sets the interestOps for the selectionKey. + */ + private void handshakeFinished() throws IOException { + // SSLEngine.getHandshakeStatus is transient and it doesn't record FINISHED status properly. + // It can move from FINISHED status to NOT_HANDSHAKING after the handshake is completed. + // Hence we also need to check handshakeResult.getHandshakeStatus() if the handshake finished or not + if (handshakeResult.getHandshakeStatus() == HandshakeStatus.FINISHED) { + //we are complete if we have delivered the last package + handshakeComplete = !netWriteBuffer.hasRemaining(); + //remove OP_WRITE if we are complete, otherwise we still have data to write + if (!handshakeComplete) + key.interestOps(key.interestOps() | SelectionKey.OP_WRITE); + else + key.interestOps(key.interestOps() & ~SelectionKey.OP_WRITE); + + log.trace("SSLHandshake FINISHED channelId {}, appReadBuffer pos {}, netReadBuffer pos {}, netWriteBuffer pos {} ", + channelId, appReadBuffer.position(), netReadBuffer.position(), netWriteBuffer.position()); + } else { + throw new IOException("NOT_HANDSHAKING during handshake"); + } + } + + /** + * Performs the WRAP function + * @param doWrite boolean + * @return SSLEngineResult + * @throws IOException + */ + private SSLEngineResult handshakeWrap(boolean doWrite) throws IOException { + log.trace("SSLHandshake handshakeWrap {}", channelId); + if (netWriteBuffer.hasRemaining()) + throw new IllegalStateException("handshakeWrap called with netWriteBuffer not empty"); + //this should never be called with a network buffer that contains data + //so we can clear it here. + netWriteBuffer.clear(); + SSLEngineResult result = sslEngine.wrap(emptyBuf, netWriteBuffer); + //prepare the results to be written + netWriteBuffer.flip(); + handshakeStatus = result.getHandshakeStatus(); + if (result.getStatus() == SSLEngineResult.Status.OK && + result.getHandshakeStatus() == HandshakeStatus.NEED_TASK) { + handshakeStatus = runDelegatedTasks(); + } + + if (doWrite) flush(netWriteBuffer); + return result; + } + + /** + * Perform handshake unwrap + * @param doRead boolean + * @return SSLEngineResult + * @throws IOException + */ + private SSLEngineResult handshakeUnwrap(boolean doRead) throws IOException { + log.trace("SSLHandshake handshakeUnwrap {}", channelId); + SSLEngineResult result; + boolean cont = false; + int read = 0; + if (doRead) { + read = socketChannel.read(netReadBuffer); + if (read == -1) throw new EOFException("EOF during handshake."); + } + do { + //prepare the buffer with the incoming data + netReadBuffer.flip(); + result = sslEngine.unwrap(netReadBuffer, appReadBuffer); + netReadBuffer.compact(); + handshakeStatus = result.getHandshakeStatus(); + if (result.getStatus() == SSLEngineResult.Status.OK && + result.getHandshakeStatus() == HandshakeStatus.NEED_TASK) { + handshakeStatus = runDelegatedTasks(); + } + cont = result.getStatus() == SSLEngineResult.Status.OK && + handshakeStatus == HandshakeStatus.NEED_UNWRAP; + log.trace("SSLHandshake handshakeUnwrap: handshakeStatus {} status {}", handshakeStatus, result.getStatus()); + } while (netReadBuffer.position() != 0 && cont); + + return result; + } + + + /** + * Reads a sequence of bytes from this channel into the given buffer. + * + * @param dst The buffer into which bytes are to be transferred + * @return The number of bytes read, possible zero or -1 if the channel has reached end-of-stream + * @throws IOException if some other I/O error occurs + */ + @Override + public int read(ByteBuffer dst) throws IOException { + if (closing) return -1; + int read = 0; + if (!handshakeComplete) return read; + + //if we have unread decrypted data in appReadBuffer read that into dst buffer. + if (appReadBuffer.position() > 0) { + read = readFromAppBuffer(dst); + } + + if (dst.remaining() > 0) { + netReadBuffer = Utils.ensureCapacity(netReadBuffer, netReadBufferSize()); + if (netReadBuffer.remaining() > 0) { + int netread = socketChannel.read(netReadBuffer); + if (netread == 0) return netread; + else if (netread < 0) throw new EOFException("EOF during read"); + } + do { + netReadBuffer.flip(); + SSLEngineResult unwrapResult = sslEngine.unwrap(netReadBuffer, appReadBuffer); + netReadBuffer.compact(); + // handle ssl renegotiation. + if (unwrapResult.getHandshakeStatus() != HandshakeStatus.NOT_HANDSHAKING) { + log.trace("SSLChannel Read begin renegotiation channelId {}, appReadBuffer pos {}, netReadBuffer pos {}, netWriteBuffer pos {}", + channelId, appReadBuffer.position(), netReadBuffer.position(), netWriteBuffer.position()); + handshake(); + break; + } + + if (unwrapResult.getStatus() == Status.OK) { + read += readFromAppBuffer(dst); + } else if (unwrapResult.getStatus() == Status.BUFFER_OVERFLOW) { + int currentApplicationBufferSize = applicationBufferSize(); + appReadBuffer = Utils.ensureCapacity(appReadBuffer, currentApplicationBufferSize); + if (appReadBuffer.position() >= currentApplicationBufferSize) { + throw new IllegalStateException("Buffer overflow when available data size (" + appReadBuffer.position() + + ") >= application buffer size (" + currentApplicationBufferSize + ")"); + } + + // appReadBuffer will extended upto currentApplicationBufferSize + // we need to read the existing content into dst before we can do unwrap again. If there are no space in dst + // we can break here. + if (dst.hasRemaining()) + read += readFromAppBuffer(dst); + else + break; + } else if (unwrapResult.getStatus() == Status.BUFFER_UNDERFLOW) { + int currentNetReadBufferSize = netReadBufferSize(); + netReadBuffer = Utils.ensureCapacity(netReadBuffer, currentNetReadBufferSize); + if (netReadBuffer.position() >= currentNetReadBufferSize) { + throw new IllegalStateException("Buffer underflow when available data size (" + netReadBuffer.position() + + ") > packet buffer size (" + currentNetReadBufferSize + ")"); + } + break; + } else if (unwrapResult.getStatus() == Status.CLOSED) { + throw new EOFException(); + } + } while (netReadBuffer.position() != 0); + } + return read; + } + + + /** + * Reads a sequence of bytes from this channel into the given buffers. + * + * @param dsts - The buffers into which bytes are to be transferred. + * @return The number of bytes read, possibly zero, or -1 if the channel has reached end-of-stream. + * @throws IOException if some other I/O error occurs + */ + @Override + public long read(ByteBuffer[] dsts) throws IOException { + return read(dsts, 0, dsts.length); + } + + + /** + * Reads a sequence of bytes from this channel into a subsequence of the given buffers. + * @param dsts - The buffers into which bytes are to be transferred + * @param offset - The offset within the buffer array of the first buffer into which bytes are to be transferred; must be non-negative and no larger than dsts.length. + * @param length - The maximum number of buffers to be accessed; must be non-negative and no larger than dsts.length - offset + * @return The number of bytes read, possibly zero, or -1 if the channel has reached end-of-stream. + * @throws IOException if some other I/O error occurs + */ + @Override + public long read(ByteBuffer[] dsts, int offset, int length) throws IOException { + if ((offset < 0) || (length < 0) || (offset > dsts.length - length)) + throw new IndexOutOfBoundsException(); + + int totalRead = 0; + int i = offset; + while (i < length) { + if (dsts[i].hasRemaining()) { + int read = read(dsts[i]); + if (read > 0) + totalRead += read; + else + break; + } + if (!dsts[i].hasRemaining()) { + i++; + } + } + return totalRead; + } + + + /** + * Writes a sequence of bytes to this channel from the given buffer. + * + * @param src The buffer from which bytes are to be retrieved + * @return The number of bytes read, possibly zero, or -1 if the channel has reached end-of-stream + * @throws IOException If some other I/O error occurs + */ + @Override + public int write(ByteBuffer src) throws IOException { + int written = 0; + if (closing) throw new IllegalStateException("Channel is in closing state"); + if (!handshakeComplete) return written; + + if (!flush(netWriteBuffer)) + return written; + + netWriteBuffer.clear(); + SSLEngineResult wrapResult = sslEngine.wrap(src, netWriteBuffer); + netWriteBuffer.flip(); + + //handle ssl renegotiation + if (wrapResult.getHandshakeStatus() != HandshakeStatus.NOT_HANDSHAKING) { + handshake(); + return written; + } + + if (wrapResult.getStatus() == Status.OK) { + written = wrapResult.bytesConsumed(); + flush(netWriteBuffer); + } else if (wrapResult.getStatus() == Status.BUFFER_OVERFLOW) { + int currentNetWriteBufferSize = netWriteBufferSize(); + netWriteBuffer.compact(); + netWriteBuffer = Utils.ensureCapacity(netWriteBuffer, currentNetWriteBufferSize); + netWriteBuffer.flip(); + if (netWriteBuffer.limit() >= currentNetWriteBufferSize) + throw new IllegalStateException("SSL BUFFER_OVERFLOW when available data size (" + netWriteBuffer.limit() + ") >= network buffer size (" + currentNetWriteBufferSize + ")"); + } else if (wrapResult.getStatus() == Status.BUFFER_UNDERFLOW) { + throw new IllegalStateException("SSL BUFFER_UNDERFLOW during write"); + } else if (wrapResult.getStatus() == Status.CLOSED) { + throw new EOFException(); + } + return written; + } + + /** + * Writes a sequence of bytes to this channel from the subsequence of the given buffers. + * + * @param srcs The buffers from which bytes are to be retrieved + * @param offset The offset within the buffer array of the first buffer from which bytes are to be retrieved; must be non-negative and no larger than srcs.length. + * @param length - The maximum number of buffers to be accessed; must be non-negative and no larger than srcs.length - offset. + * @return returns no.of bytes written , possibly zero. + * @throws IOException If some other I/O error occurs + */ + @Override + public long write(ByteBuffer[] srcs, int offset, int length) throws IOException { + if ((offset < 0) || (length < 0) || (offset > srcs.length - length)) + throw new IndexOutOfBoundsException(); + int totalWritten = 0; + int i = offset; + while (i < length) { + if (srcs[i].hasRemaining() || hasPendingWrites()) { + int written = write(srcs[i]); + if (written > 0) { + totalWritten += written; + } + } + if (!srcs[i].hasRemaining() && !hasPendingWrites()) { + i++; + } else { + // if we are unable to write the current buffer to socketChannel we should break, + // as we might have reached max socket send buffer size. + break; + } + } + return totalWritten; + } + + /** + * Writes a sequence of bytes to this channel from the given buffers. + * + * @param srcs The buffers from which bytes are to be retrieved + * @return returns no.of bytes consumed by SSLEngine.wrap , possibly zero. + * @throws IOException If some other I/O error occurs + */ + @Override + public long write(ByteBuffer[] srcs) throws IOException { + return write(srcs, 0, srcs.length); + } + + + /** + * SSLSession's peerPrincipal for the remote host. + * @return Principal + */ + public Principal peerPrincipal() throws IOException { + try { + return sslEngine.getSession().getPeerPrincipal(); + } catch (SSLPeerUnverifiedException se) { + log.warn("SSL peer is not authenticated, returning ANONYMOUS instead"); + return KafkaPrincipal.ANONYMOUS; + } + } + + /** + * returns a SSL Session after the handshake is established + * throws IllegalStateException if the handshake is not established + */ + public SSLSession sslSession() throws IllegalStateException { + return sslEngine.getSession(); + } + + /** + * Adds interestOps to SelectionKey of the TransportLayer + * @param ops SelectionKey interestOps + */ + @Override + public void addInterestOps(int ops) { + if (!key.isValid()) + throw new CancelledKeyException(); + else if (!handshakeComplete) + throw new IllegalStateException("handshake is not completed"); + + key.interestOps(key.interestOps() | ops); + } + + /** + * removes interestOps to SelectionKey of the TransportLayer + * @param ops SelectionKey interestOps + */ + @Override + public void removeInterestOps(int ops) { + if (!key.isValid()) + throw new CancelledKeyException(); + else if (!handshakeComplete) + throw new IllegalStateException("handshake is not completed"); + + key.interestOps(key.interestOps() & ~ops); + } + + + /** + * returns delegatedTask for the SSLEngine. + */ + protected Runnable delegatedTask() { + return sslEngine.getDelegatedTask(); + } + + /** + * transfers appReadBuffer contents (decrypted data) into dst bytebuffer + * @param dst ByteBuffer + */ + private int readFromAppBuffer(ByteBuffer dst) { + appReadBuffer.flip(); + int remaining = Math.min(appReadBuffer.remaining(), dst.remaining()); + if (remaining > 0) { + int limit = appReadBuffer.limit(); + appReadBuffer.limit(appReadBuffer.position() + remaining); + dst.put(appReadBuffer); + appReadBuffer.limit(limit); + } + appReadBuffer.compact(); + return remaining; + } + + protected int netReadBufferSize() { + return sslEngine.getSession().getPacketBufferSize(); + } + + protected int netWriteBufferSize() { + return sslEngine.getSession().getPacketBufferSize(); + } + + protected int applicationBufferSize() { + return sslEngine.getSession().getApplicationBufferSize(); + } + + protected ByteBuffer netReadBuffer() { + return netReadBuffer; + } + + private void handshakeFailure() { + //Release all resources such as internal buffers that SSLEngine is managing + sslEngine.closeOutbound(); + try { + sslEngine.closeInbound(); + } catch (SSLException e) { + log.debug("SSLEngine.closeInBound() raised an exception.", e); + } + } + + @Override + public boolean isMute() { + return key.isValid() && (key.interestOps() & SelectionKey.OP_READ) == 0; + } + + @Override + public long transferFrom(FileChannel fileChannel, long position, long count) throws IOException { + return fileChannel.transferTo(position, count, this); + } + +} 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 b68bbf00ab8eb..10ca632357ae7 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,21 +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 + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this 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.network; + import java.io.IOException; import java.net.InetSocketAddress; import java.util.List; @@ -25,6 +22,11 @@ */ public interface Selectable { + /** + * See {@link #connect(String, InetSocketAddress, int, int) connect()} + */ + public static final int USE_DEFAULT_BUFFER_SIZE = -1; + /** * Begin establishing a socket connection to the given address identified by the given address * @param id The id for this connection @@ -33,12 +35,7 @@ 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; - - /** - * Begin disconnecting the connection identified by the given id - */ - public void disconnect(int id); + public void connect(String id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException; /** * Wakeup this selector if it is blocked on I/O @@ -51,34 +48,70 @@ 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) + * Close the connection identified by the given id + */ + public void close(String id); + + /** + * Queue the given request for sending in the subsequent {@link #poll(long) poll()} 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) 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) 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) 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) poll()} * call. */ - public List connected(); + public List connected(); -} \ No newline at end of file + /** + * 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(); + + /** + * returns true if a channel is ready + * @param id The id for the connection + */ + public boolean isChannelReady(String id); +} 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 4dd2cdf773f7e..e1e5b4a3e6192 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 @@ -3,9 +3,9 @@ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the * License. You may obtain a copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the * specific language governing permissions and limitations under the License. @@ -13,98 +13,120 @@ package org.apache.kafka.common.network; import java.io.IOException; -import java.net.InetAddress; 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 * the connection. The successful invocation of this method does not mean a valid connection has been established. - * + * * Sending requests, receiving responses, processing connection completions, and disconnections on the existing * connections are all done using the poll() call. - * + * *
            - * List<NetworkRequest> requestsToSend = Arrays.asList(new NetworkRequest(0, myBytes), new NetworkRequest(1, myOtherBytes));
            - * selector.poll(TIMEOUT_MS, requestsToSend);
            + * nioSelector.send(new NetworkSend(myDestination, myBytes));
            + * nioSelector.send(new NetworkSend(myOtherDestination, myOtherBytes));
            + * nioSelector.poll(TIMEOUT_MS);
              * 
            - * - * 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 static final Logger log = LoggerFactory.getLogger(Selector.class); - private final java.nio.channels.Selector selector; - private final Map keys; - private final List completedSends; + private final java.nio.channels.Selector nioSelector; + private final Map channels; + private final List completedSends; private final List completedReceives; - private final List disconnected; - private final List connected; + private final Map> stagedReceives; + 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 ChannelBuilder channelBuilder; + 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(Metrics metrics, Time time) { + public Selector(int maxReceiveSize, long connectionMaxIdleMs, Metrics metrics, Time time, String metricGrpPrefix, Map metricTags, boolean metricsPerConnection, ChannelBuilder channelBuilder) { try { - this.selector = java.nio.channels.Selector.open(); + this.nioSelector = java.nio.channels.Selector.open(); } catch (IOException e) { throw new KafkaException(e); } + this.maxReceiveSize = maxReceiveSize; + this.connectionsMaxIdleNanos = connectionMaxIdleMs * 1000 * 1000; this.time = time; - this.keys = new HashMap(); - this.completedSends = new ArrayList(); + this.metricGrpPrefix = metricGrpPrefix; + this.metricTags = metricTags; + this.channels = new HashMap(); + this.completedSends = new ArrayList(); this.completedReceives = new ArrayList(); - this.connected = new ArrayList(); - this.disconnected = new ArrayList(); + this.stagedReceives = new HashMap>(); + this.connected = new ArrayList(); + this.disconnected = new ArrayList(); + this.failedSends = new ArrayList(); this.sensors = new SelectorMetrics(metrics); + this.channelBuilder = channelBuilder; + // 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, ChannelBuilder channelBuilder) { + this(NetworkReceive.UNLIMITED, connectionMaxIdleMS, metrics, time, metricGrpPrefix, metricTags, true, channelBuilder); } /** - * 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)} + * Note that this call only initiates the connection, which will be completed on a future {@link #poll(long)} * call. Check {@link #connected()} to see which (if any) connections have completed after a given poll call. * @param id The id for the new connection * @param address The address to connect to @@ -114,48 +136,52 @@ public Selector(Metrics metrics, Time time) { * @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 { - if (this.keys.containsKey(id)) + public void connect(String id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException { + if (this.channels.containsKey(id)) throw new IllegalStateException("There is already a connection for id " + id); - SocketChannel channel = SocketChannel.open(); - channel.configureBlocking(false); - Socket socket = channel.socket(); + SocketChannel socketChannel = SocketChannel.open(); + socketChannel.configureBlocking(false); + Socket socket = socketChannel.socket(); socket.setKeepAlive(true); - socket.setSendBufferSize(sendBufferSize); - socket.setReceiveBufferSize(receiveBufferSize); + if (sendBufferSize != Selectable.USE_DEFAULT_BUFFER_SIZE) + socket.setSendBufferSize(sendBufferSize); + if (receiveBufferSize != Selectable.USE_DEFAULT_BUFFER_SIZE) + socket.setReceiveBufferSize(receiveBufferSize); socket.setTcpNoDelay(true); try { - channel.connect(address); + socketChannel.connect(address); } catch (UnresolvedAddressException e) { - channel.close(); + socketChannel.close(); throw new IOException("Can't resolve address: " + address, e); } catch (IOException e) { - channel.close(); + socketChannel.close(); throw e; } - SelectionKey key = channel.register(this.selector, SelectionKey.OP_CONNECT); - key.attach(new Transmissions(id)); - this.keys.put(id, key); + SelectionKey key = socketChannel.register(nioSelector, SelectionKey.OP_CONNECT); + KafkaChannel channel = channelBuilder.buildChannel(id, key, maxReceiveSize); + key.attach(channel); + this.channels.put(id, channel); } /** - * Disconnect any connections for the given id (if there are any). The disconnection is asynchronous and will not be - * processed until the next {@link #poll(long, List) poll()} call. + * 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 */ - @Override - public void disconnect(int id) { - SelectionKey key = this.keys.get(id); - if (key != null) - key.cancel(); + public void register(String id, SocketChannel socketChannel) throws ClosedChannelException { + SelectionKey key = socketChannel.register(nioSelector, SelectionKey.OP_READ); + KafkaChannel channel = channelBuilder.buildChannel(id, key, maxReceiveSize); + key.attach(channel); + this.channels.put(id, channel); } /** - * 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(); } /** @@ -163,121 +189,146 @@ public void wakeup() { */ @Override public void close() { - for (SelectionKey key : this.selector.keys()) - close(key); + List connections = new ArrayList<>(channels.keySet()); + for (String id : connections) + close(id); try { - this.selector.close(); + this.nioSelector.close(); } catch (IOException e) { - log.error("Exception closing selector:", e); + log.error("Exception closing nioSelector:", e); + } catch (SecurityException se) { + log.error("Exception closing nioSelector:", se); + } + sensors.close(); + } + + /** + * Queue the given request for sending in the subsequent {@poll(long)} calls + * @param send The request to send + */ + public void send(Send send) { + KafkaChannel channel = channelOrFail(send.destination()); + try { + channel.setSend(send); + } catch (CancelledKeyException e) { + this.failedSends.add(send.destination()); + close(channel); } } /** * 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 - * lists will be cleared at the beginning of each {@link #poll(long, List)} call and repopulated by the call if any - * 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 - * + * lists will be cleared at the beginning of each {@link #poll(long)} call and repopulated by the call if there is + * any completed I/O. + * + * In the "Plaintext" setting, we are using socketChannel to read & write to the network. But for the "SSL" setting, + * we encrypt the data before we use socketChannel to write data to the network, and decrypt before we return the responses. + * This requires additional buffers to be maintained as we are reading from network, since the data on the wire is encrpyted + * we won't be able to read exact no.of bytes as kafka protocol requires. We read as many bytes as we can, up to SSLEngine's + * application buffer size. This means we might be reading additional bytes than the requested size. + * If there is no further data to read from socketChannel selector won't invoke that channel and we've have additional bytes + * in the buffer. To overcome this issue we added "stagedReceives" map which contains per-channel deque. When we are + * reading a channel we read as many responses as we can and store them into "stagedReceives" and pop one response during + * the poll to add the completedReceives. If there are any active channels in the "stagedReceives" we set "timeout" to 0 + * and pop response and add to the completedReceives. + * + * @param timeout The amount of time to wait, in milliseconds, which must be non-negative + * @throws IllegalArgumentException If `timeout` is negative * @throws IllegalStateException If a send is given for which we have no existing connection or for which there is * already an in-progress send + * @throws InvalidReceiveException If invalid data is received */ @Override - public void poll(long timeout, List sends) throws IOException { + public void poll(long timeout) throws IOException { + if (timeout < 0) + throw new IllegalArgumentException("timeout should be >= 0"); 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); - } - } - + if (hasStagedReceives()) + timeout = 0; /* 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(); iter.remove(); + KafkaChannel channel = channel(key); - Transmissions transmissions = transmissions(key); - SocketChannel channel = channel(key); - - // register all per-broker metrics at once - sensors.maybeRegisterNodeMetrics(transmissions.id); + // register all per-connection metrics at once + sensors.maybeRegisterConnectionMetrics(channel.id()); + lruConnections.put(channel.id(), currentTimeNanos); try { /* 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.connected.add(channel.id()); this.sensors.connectionCreated.record(); } - /* read from any connections that have readable data */ - if (key.isReadable()) { - if (!transmissions.hasReceive()) - transmissions.receive = new NetworkReceive(transmissions.id); - transmissions.receive.readFrom(channel); - if (transmissions.receive.complete()) { - transmissions.receive.payload().rewind(); - this.completedReceives.add(transmissions.receive); - this.sensors.recordBytesReceived(transmissions.id, transmissions.receive.payload().limit()); - transmissions.clearReceive(); + /* if channel is not ready finish prepare */ + if (channel.isConnected() && !channel.ready()) + channel.prepare(); + + /* if channel is ready read from any connections that have readable data */ + if (channel.ready() && key.isReadable() && !hasStagedReceive(channel)) { + NetworkReceive networkReceive; + try { + while ((networkReceive = channel.read()) != null) { + addToStagedReceives(channel, networkReceive); + } + } catch (InvalidReceiveException e) { + log.error("Invalid data received from " + channel.id() + " closing connection", e); + close(channel); + this.disconnected.add(channel.id()); + throw e; } } - /* 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) { - this.completedSends.add(transmissions.send); - this.sensors.recordBytesSent(transmissions.id, transmissions.send.size()); - transmissions.clearSend(); - key.interestOps(key.interestOps() & ~SelectionKey.OP_WRITE); + /* if channel is ready write to any sockets that have space in their buffer and for which we have data */ + if (channel.ready() && key.isWritable()) { + Send send = channel.write(); + if (send != null) { + this.completedSends.add(send); + this.sensors.recordBytesSent(channel.id(), send.size()); } } /* cancel any defunct sockets */ - if (!key.isValid()) - close(key); + if (!key.isValid()) { + close(channel); + this.disconnected.add(channel.id()); + } } catch (IOException e) { - InetAddress remoteAddress = null; - Socket socket = channel.socket(); - if (socket != null) - remoteAddress = socket.getInetAddress(); - log.warn("Error in I/O with {}", remoteAddress , e); - close(key); + String desc = channel.socketDescription(); + log.debug("Connection with {} disconnected", desc, e); + close(channel); + this.disconnected.add(channel.id()); } } } + + addToCompletedReceives(); + long endIo = time.nanoseconds(); this.sensors.ioTime.record(endIo - endSelect, time.milliseconds()); + maybeCloseOldestConnection(); } + + @Override - public List completedSends() { + public List completedSends() { return this.completedSends; } @@ -287,15 +338,68 @@ 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) { + KafkaChannel channel = channelOrFail(id); + mute(channel); + } + + private void mute(KafkaChannel channel) { + channel.mute(); + } + + @Override + public void unmute(String id) { + KafkaChannel channel = channelOrFail(id); + unmute(channel); + } + + private void unmute(KafkaChannel channel) { + channel.unmute(); + } + + @Override + public void muteAll() { + for (KafkaChannel channel : this.channels.values()) + mute(channel); + } + + @Override + public void unmuteAll() { + for (KafkaChannel channel : this.channels.values()) + unmute(channel); + } + + 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 */ @@ -304,100 +408,145 @@ private void clear() { this.completedReceives.clear(); this.connected.clear(); this.disconnected.clear(); + this.disconnected.addAll(this.failedSends); + this.failedSends.clear(); } /** * Check for data, waiting up to the given timeout. - * - * @param ms Length of time to wait, in milliseconds. If negative, wait indefinitely. + * + * @param ms Length of time to wait, in milliseconds, which must be non-negative * @return The number of keys ready + * @throws IllegalArgumentException * @throws IOException */ private int select(long ms) throws IOException { + if (ms < 0L) + throw new IllegalArgumentException("timeout should be >= 0"); + if (ms == 0L) - return this.selector.selectNow(); - else if (ms < 0L) - return this.selector.select(); + return this.nioSelector.selectNow(); else - return this.selector.select(ms); + return this.nioSelector.select(ms); + } + + /** + * Close the connection identified by the given id + */ + public void close(String id) { + KafkaChannel channel = this.channels.get(id); + if (channel != null) + close(channel); } /** * Begin closing this connection */ - private void close(SelectionKey key) { - SocketChannel channel = channel(key); - Transmissions trans = transmissions(key); - if (trans != null) { - this.disconnected.add(trans.id); - this.keys.remove(trans.id); - trans.clearReceive(); - trans.clearSend(); - } - key.attach(null); - key.cancel(); + private void close(KafkaChannel channel) { try { - channel.socket().close(); channel.close(); } catch (IOException e) { - log.error("Exception closing connection to node {}:", trans.id, e); + log.error("Exception closing connection to node {}:", channel.id(), e); } + this.stagedReceives.remove(channel); + this.channels.remove(channel.id()); + this.lruConnections.remove(channel.id()); this.sensors.connectionClosed.record(); } + /** - * Get the selection key associated with this numeric id + * check if channel is ready */ - private SelectionKey keyForId(int 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."); - return key; + @Override + public boolean isChannelReady(String id) { + KafkaChannel channel = this.channels.get(id); + if (channel == null) + return false; + return channel.ready(); + } + + private KafkaChannel channelOrFail(String id) { + KafkaChannel channel = this.channels.get(id); + if (channel == null) + throw new IllegalStateException("Attempt to retrieve channel for which there is no open connection. Connection id " + id + " existing connections " + channels.keySet().toString()); + return channel; } /** - * Get the transmissions for the given connection + * Return the selector channels. */ - private Transmissions transmissions(SelectionKey key) { - return (Transmissions) key.attachment(); + public List channels() { + return new ArrayList<>(channels.values()); } /** - * Get the socket channel associated with this selection key + * Return the channel associated with this connection or `null` if there is no channel associated with the + * connection. */ - private SocketChannel channel(SelectionKey key) { - return (SocketChannel) key.channel(); + public KafkaChannel channel(String id) { + return this.channels.get(id); } /** - * The id and in-progress send and receive associated with a connection + * Get the channel associated with selectionKey */ - private static class Transmissions { - public int id; - public NetworkSend send; - public NetworkReceive receive; + private KafkaChannel channel(SelectionKey key) { + return (KafkaChannel) key.attachment(); + } - public Transmissions(int id) { - this.id = id; - } + /** + * Check if given channel has a staged receive + */ + private boolean hasStagedReceive(KafkaChannel channel) { + return stagedReceives.containsKey(channel); + } - public boolean hasSend() { - return this.send != null; + /** + * check if stagedReceives have unmuted channel + */ + private boolean hasStagedReceives() { + for (KafkaChannel channel : this.stagedReceives.keySet()) { + if (!channel.isMute()) + return true; } + return false; + } - public void clearSend() { - this.send = null; - } - public boolean hasReceive() { - return this.receive != null; - } + /** + * adds a receive to staged receieves + */ + private void addToStagedReceives(KafkaChannel channel, NetworkReceive receive) { + if (!stagedReceives.containsKey(channel)) + stagedReceives.put(channel, new ArrayDeque()); - public void clearReceive() { - this.receive = null; + Deque deque = stagedReceives.get(channel); + deque.add(receive); + } + + /** + * checks if there are any staged receives and adds to completedReceives + */ + private void addToCompletedReceives() { + if (this.stagedReceives.size() > 0) { + Iterator>> iter = this.stagedReceives.entrySet().iterator(); + while (iter.hasNext()) { + Map.Entry> entry = iter.next(); + KafkaChannel channel = entry.getKey(); + if (!channel.isMute()) { + Deque deque = entry.getValue(); + NetworkReceive networkReceive = deque.poll(); + this.completedReceives.add(networkReceive); + this.sensors.recordBytesReceived(channel.id(), networkReceive.payload().limit()); + if (deque.size() == 0) + iter.remove(); + } + } } } + private class SelectorMetrics { private final Metrics metrics; public final Sensor connectionClosed; @@ -408,101 +557,145 @@ private class SelectorMetrics { public final Sensor selectTime; public final Sensor ioTime; + /* Names of metrics that are not registered through sensors */ + private final List topLevelMetricNames = new ArrayList<>(); + private final List sensors = new ArrayList<>(); + 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"); - this.connectionClosed.add("connection-close-rate", "Connections closed per second in the window.", new Rate()); - - this.connectionCreated = this.metrics.sensor("connections-created"); - this.connectionCreated.add("connection-creation-rate", "New connections established per second in the window.", new Rate()); - - this.bytesTransferred = this.metrics.sensor("bytes-sent-received"); - bytesTransferred.add("network-io-rate", - "The average number of network operations (reads or writes) on all connections per second.", - new Rate(new Count())); - - this.bytesSent = this.metrics.sensor("bytes-sent", bytesTransferred); - this.bytesSent.add("outgoing-byte-rate", "The average number of outgoing bytes sent per second to all servers.", new Rate()); - this.bytesSent.add("request-rate", "The average number of requests sent per second.", new Rate(new Count())); - this.bytesSent.add("request-size-avg", "The average size of all requests in the window..", new Avg()); - this.bytesSent.add("request-size-max", "The maximum size of any request sent in the window.", new Max()); - - this.bytesReceived = this.metrics.sensor("bytes-received", bytesTransferred); - this.bytesReceived.add("incoming-byte-rate", "Bytes/second read off all sockets", new Rate()); - this.bytesReceived.add("response-rate", "Responses received sent per second.", new Rate(new Count())); - - this.selectTime = this.metrics.sensor("select-time"); - this.selectTime.add("select-rate", - "Number of times the I/O layer checked for new I/O to perform per second", - new Rate(new Count())); - this.selectTime.add("io-wait-time-ns-avg", - "The average length of time the I/O thread spent waiting for a socket ready for reads or writes in nanoseconds.", - new Avg()); - this.selectTime.add("io-wait-ratio", "The fraction of time the I/O thread spent waiting.", new Rate(TimeUnit.NANOSECONDS)); - - this.ioTime = this.metrics.sensor("io-time"); - this.ioTime.add("io-time-ns-avg", "The average length of time for I/O per select call in nanoseconds.", new Avg()); - this.ioTime.add("io-ratio", "The fraction of time the I/O thread spent doing I/O", new Rate(TimeUnit.NANOSECONDS)); - - this.metrics.addMetric("connection-count", "The current number of active connections.", new Measurable() { + this.connectionClosed = 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 = 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 = 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 = 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 = 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 = 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 = 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); + topLevelMetricNames.add(metricName); + this.metrics.addMetric(metricName, new Measurable() { public double measure(MetricConfig config, long now) { - return keys.size(); + return channels.size(); } }); } - public void maybeRegisterNodeMetrics(int node) { - if (node >= 0) { - // if one sensor of the metrics has been registered for the node, + private Sensor sensor(String name, Sensor... parents) { + Sensor sensor = metrics.sensor(name, parents); + sensors.add(sensor); + return sensor; + } + + 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-" + node + ".bytes-sent"; + String nodeRequestName = "node-" + connectionId + ".bytes-sent"; Sensor nodeRequest = this.metrics.getSensor(nodeRequestName); if (nodeRequest == null) { - nodeRequest = this.metrics.sensor(nodeRequestName); - nodeRequest.add("node-" + node + ".outgoing-byte-rate", new Rate()); - nodeRequest.add("node-" + node + ".request-rate", - "The average number of requests sent per second.", - new Rate(new Count())); - nodeRequest.add("node-" + node + ".request-size-avg", "The average size of all requests in the window..", new Avg()); - nodeRequest.add("node-" + node + ".request-size-max", "The maximum size of any request sent in the window.", new Max()); - - String nodeResponseName = "node-" + node + ".bytes-received"; - Sensor nodeResponse = this.metrics.sensor(nodeResponseName); - nodeResponse.add("node-" + node + ".incoming-byte-rate", new Rate()); - nodeResponse.add("node-" + node + ".response-rate", - "The average number of responses received per second.", - new Rate(new Count())); - - String nodeTimeName = "node-" + node + ".latency"; - Sensor nodeRequestTime = this.metrics.sensor(nodeTimeName); - nodeRequestTime.add("node-" + node + ".request-latency-avg", new Avg()); - nodeRequestTime.add("node-" + node + ".request-latency-max", new Max()); + String metricGrpName = metricGrpPrefix + "-node-metrics"; + + Map tags = new LinkedHashMap(metricTags); + tags.put("node-id", "node-" + connectionId); + + nodeRequest = 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 = 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 = 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(int node, int bytes) { + public void recordBytesSent(String connectionId, long bytes) { long now = time.milliseconds(); this.bytesSent.record(bytes, now); - if (node >= 0) { - String nodeRequestName = "node-" + node + ".bytes-sent"; + 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(int node, int bytes) { + public void recordBytesReceived(String connection, int bytes) { long now = time.milliseconds(); this.bytesReceived.record(bytes, now); - if (node >= 0) { - String nodeRequestName = "node-" + node + ".bytes-received"; + if (!connection.isEmpty()) { + String nodeRequestName = "node-" + connection + ".bytes-received"; Sensor nodeRequest = this.metrics.getSensor(nodeRequestName); if (nodeRequest != null) nodeRequest.record(bytes, now); } } + + public void close() { + for (MetricName metricName : topLevelMetricNames) + metrics.removeMetric(metricName); + for (Sensor sensor : sensors) + metrics.removeSensor(sensor.name()); + } } } 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 5d321a09e4701..e0d883125e6d0 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 @@ -3,9 +3,9 @@ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the * License. You may obtain a copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the * specific language governing permissions and limitations under the License. @@ -13,7 +13,6 @@ package org.apache.kafka.common.network; import java.io.IOException; -import java.nio.ByteBuffer; import java.nio.channels.GatheringByteChannel; /** @@ -24,30 +23,25 @@ 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 completed(); - /** - * An optional method to turn this send into an array of ByteBuffers if possible (otherwise returns null) - */ - public ByteBuffer[] reify(); - /** * Write some as-yet unwritten bytes from this send to the provided channel. It may take multiple calls for the send * to be completely written - * @param channel The channel to write to + * @param channel The Channel to write to * @return The number of bytes written * @throws IOException If the write fails */ 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/network/TransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java new file mode 100644 index 0000000000000..ff7a3bf85dd59 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java @@ -0,0 +1,104 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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; + +/* + * Transport layer for underlying communication. + * At very basic level it is wrapper around SocketChannel and can be used as substitue for SocketChannel + * and other network Channel implementations. + * As NetworkClient replaces BlockingChannel and other implementations we will be using KafkaChannel as + * a network I/O channel. + */ +import java.io.IOException; +import java.nio.channels.FileChannel; +import java.nio.channels.SocketChannel; +import java.nio.channels.ScatteringByteChannel; +import java.nio.channels.GatheringByteChannel; + +import java.security.Principal; + + +public interface TransportLayer extends ScatteringByteChannel, GatheringByteChannel { + + /** + * Returns true if the channel has handshake and authentication done. + */ + boolean ready(); + + /** + * Finishes the process of connecting a socket channel. + */ + void finishConnect() throws IOException; + + /** + * disconnect socketChannel + */ + void disconnect(); + + /** + * Tells whether or not this channel's network socket is connected. + */ + boolean isConnected(); + + /** + * returns underlying socketChannel + */ + SocketChannel socketChannel(); + + + /** + * Performs SSL handshake hence is a no-op for the non-secure + * implementation + * @throws IOException + */ + void handshake() throws IOException; + + /** + * Returns true if there are any pending writes + */ + boolean hasPendingWrites(); + + /** + * returns SSLSession.getPeerPrinicpal if SSLTransportLayer used + * for non-secure returns a "ANONYMOUS" as the peerPrincipal + */ + Principal peerPrincipal() throws IOException; + + void addInterestOps(int ops); + + void removeInterestOps(int ops); + + boolean isMute(); + + /** + * Transfers bytes from `fileChannel` to this `TransportLayer`. + * + * This method will delegate to {@link FileChannel#transferTo(long, long, java.nio.channels.WritableByteChannel)}, + * but it will unwrap the destination channel, if possible, in order to benefit from zero copy. This is required + * because the fast path of `transferTo` is only executed if the destination buffer inherits from an internal JDK + * class. + * + * @param fileChannel The source channel + * @param position The position within the file at which the transfer is to begin; must be non-negative + * @param count The maximum number of bytes to be transferred; must be non-negative + * @return The number of bytes, possibly zero, that were actually transferred + * @see FileChannel#transferTo(long, long, java.nio.channels.WritableByteChannel) + */ + long transferFrom(FileChannel fileChannel, long position, long count) throws IOException; + +} 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 109fc965e09b2..46ddddb2a653a 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 @@ -16,40 +16,40 @@ */ package org.apache.kafka.common.protocol; - -import java.util.ArrayList; -import java.util.List; - /** * 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(8, "offset_commit"), - OFFSET_FETCH(9, "offset_fetch"), - CONSUMER_METADATA(10, "consumer_metadata"), - JOIN_GROUP(11, "join_group"), - HEARTBEAT(12, "heartbeat"); + 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"); private static ApiKeys[] codeToType; - public static int MAX_API_KEY = -1; + 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[MAX_API_KEY+1]; + 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 */ + /** the permanent and immutable id of an API--this can't change ever */ public final short id; /** an english description of the api--this is for debugging and can change */ @@ -63,4 +63,4 @@ private ApiKeys(int id, String 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 d434f420ad634..220132f7f023d 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 @@ -20,7 +20,9 @@ import java.util.Map; import org.apache.kafka.common.errors.*; - +import org.apache.kafka.common.errors.ControllerMovedException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * This class contains all the client-server errors--those errors that must be sent from the server to the client. These @@ -31,29 +33,76 @@ 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.")), - 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.")), + OFFSET_OUT_OF_RANGE(1, + 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.")), // 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.")), - // TODO: errorCode 8, 9, 11 - 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.")), - // TODO: errorCode 14, 15, 16 - INVALID_TOPIC_EXCEPTION(17, new InvalidTopicException("The request attempted to perform an operation on an invalid topic.")); + 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.")), + BROKER_NOT_AVAILABLE(8, + new BrokerNotAvailableException("The broker is not available.")), + 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.")), + STALE_CONTROLLER_EPOCH(11, + new ControllerMovedException("The controller moved to another broker.")), + 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")), + AUTHORIZATION_FAILED(29, new ApiException("Request is not authorized.")), + REBALANCE_IN_PROGRESS(30, + new ApiException("The group is rebalancing, so a rejoin is needed.")); + + private static final Logger log = LoggerFactory.getLogger(Errors.class); 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; @@ -82,8 +131,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; + } } /** @@ -91,11 +141,16 @@ public void maybeThrow() { */ public static Errors forCode(short code) { Errors error = codeToError.get(code); - return error == null ? UNKNOWN : error; + if (error != null) { + return error; + } else { + log.warn("Unexpected error code: {}.", code); + return UNKNOWN; + } } /** - * Return the error instance associated with this exception (or UKNOWN if there is none) + * Return the error instance associated with this exception (or UNKNOWN if there is none) */ public static Errors forException(Throwable t) { Errors error = classToError.get(t.getClass()); 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 c2cbbbd7f1f28..85357abcdc2e0 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 @@ -29,6 +29,8 @@ private static Schema schemaFor(Schema[][] schemas, int apiKey, int version) { Schema[] versions = schemas[apiKey]; if (version < 0 || version > versions.length) throw new IllegalArgumentException("Invalid version for API key " + apiKey + ": " + version); + if (versions[version] == null) + throw new IllegalArgumentException("Unsupported version for API key " + apiKey + ": " + version); return versions[version]; } @@ -62,4 +64,8 @@ public static Struct parseResponse(int apiKey, ByteBuffer buffer) { return (Struct) currentResponseSchema(apiKey).read(buffer); } + public static Struct parseResponse(int apiKey, int version, ByteBuffer buffer) { + return (Struct) responseSchema(apiKey, version).read(buffer); + } + } 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 7517b879866fc..b72db4fb3b0e7 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 @@ -16,82 +16,85 @@ */ package org.apache.kafka.common.protocol; -import static org.apache.kafka.common.protocol.types.Type.BYTES; -import static org.apache.kafka.common.protocol.types.Type.INT16; -import static org.apache.kafka.common.protocol.types.Type.INT32; -import static org.apache.kafka.common.protocol.types.Type.INT64; -import static org.apache.kafka.common.protocol.types.Type.STRING; - 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 static org.apache.kafka.common.protocol.types.Type.*; + 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 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 Schema RESPONSE_HEADER = new Schema(new Field("correlation_id", - INT32, - "The user-supplied value passed in with 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 metadata 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", @@ -100,294 +103,494 @@ public class Protocol { INT16), 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_V1 = PRODUCE_REQUEST_V0; + + public static final Schema PRODUCE_RESPONSE_V1 = new Schema(new Field("responses", + new ArrayOf(new Schema(new Field("topic", STRING), + new Field("partition_responses", + new ArrayOf(new Schema(new Field("partition", + INT32), + new Field("error_code", + INT16), + new Field("base_offset", + INT64))))))), + new Field("throttle_time_ms", + INT32, + "Duration in milliseconds for which the request was throttled" + + " due to quota violation. (Zero if the request did not violate any quota.)", + 0)); + + public static final Schema[] PRODUCE_REQUEST = new Schema[] {PRODUCE_REQUEST_V0, PRODUCE_REQUEST_V1}; + public static final Schema[] PRODUCE_RESPONSE = new Schema[] {PRODUCE_RESPONSE_V0, PRODUCE_RESPONSE_V1}; /* Offset commit api */ - public static 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("timestamp", - INT64, - "Timestamp of the commit"), - new Field("metadata", - STRING, - "Any associated metadata the client wants to keep.")); - - public static 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 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 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_V0), - "Topics to commit offsets.")); - - public static Schema OFFSET_COMMIT_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", - INT32, - "Topic partition id."), - new Field("error_code", - INT16)); - - public static 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 Schema OFFSET_COMMIT_RESPONSE_V0 = new Schema(new Field("responses", - new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0))); - - public static Schema[] OFFSET_COMMIT_REQUEST = new Schema[] { OFFSET_COMMIT_REQUEST_V0, OFFSET_COMMIT_REQUEST_V1 }; - /* The response types for both V0 and V1 of OFFSET_COMMIT_REQUEST are the same. */ - public static Schema[] OFFSET_COMMIT_RESPONSE = new Schema[] { OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V0}; + 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.")); - /* Offset fetch api */ - public static Schema OFFSET_FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition", - INT32, - "Topic partition id.")); + 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 Schema OFFSET_FETCH_REQUEST_TOPIC_V0 = new Schema(new Field("topic", + public static final Schema OFFSET_COMMIT_REQUEST_V2 = new Schema(new Field("group_id", STRING, - "Topic to fetch offset."), - new Field("partitions", - new ArrayOf(OFFSET_FETCH_REQUEST_PARTITION_V0), - "Partitions to fetch offsets.")); - - public static 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 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)); + "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 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_COMMIT_RESPONSE_V0 = new Schema(new Field("responses", + new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0))); - public static Schema OFFSET_FETCH_RESPONSE_V0 = new Schema(new Field("responses", - new ArrayOf(OFFSET_FETCH_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 */ - public static Schema[] OFFSET_FETCH_REQUEST = new Schema[] { OFFSET_FETCH_REQUEST_V0 }; - public static Schema[] OFFSET_FETCH_RESPONSE = new Schema[] { OFFSET_FETCH_RESPONSE_V0 }; + /* + * 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 Schema LIST_OFFSET_REQUEST_PARTITION_V0 = new Schema(new Field("partition", + 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("timestamp", + new Field("fetch_offset", INT64, - "Timestamp."), - new Field("max_num_offsets", + "Message offset."), + new Field("max_bytes", INT32, - "Maximum offsets to return.")); + "Maximum bytes to fetch.")); - public static Schema LIST_OFFSET_REQUEST_TOPIC_V0 = new Schema(new Field("topic", - STRING, - "Topic to list offset."), + public static final Schema FETCH_REQUEST_TOPIC_V0 = new Schema(new Field("topic", STRING, "Topic to fetch."), new Field("partitions", - new ArrayOf(LIST_OFFSET_REQUEST_PARTITION_V0), - "Partitions to list offset.")); + new ArrayOf(FETCH_REQUEST_PARTITION_V0), + "Partitions to fetch.")); - public static Schema LIST_OFFSET_REQUEST_V0 = new Schema(new Field("replica_id", + 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(LIST_OFFSET_REQUEST_TOPIC_V0), - "Topics to list offsets.")); + new ArrayOf(FETCH_REQUEST_TOPIC_V0), + "Topics to fetch.")); - public static Schema LIST_OFFSET_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", + // The V1 Fetch Request body is the same as V0. + // Only the version number is incremented to indicate a newer client + public static final Schema FETCH_REQUEST_V1 = FETCH_REQUEST_V0; + public static final Schema FETCH_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.")); + new Field("error_code", INT16), + new Field("high_watermark", + INT64, + "Last committed offset."), + new Field("record_set", BYTES)); - public static Schema LIST_OFFSET_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), + public static final Schema FETCH_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), new Field("partition_responses", - new ArrayOf(LIST_OFFSET_RESPONSE_PARTITION_V0))); + new ArrayOf(FETCH_RESPONSE_PARTITION_V0))); - public static Schema LIST_OFFSET_RESPONSE_V0 = new Schema(new Field("responses", - new ArrayOf(LIST_OFFSET_RESPONSE_TOPIC_V0))); + public static final Schema FETCH_RESPONSE_V0 = new Schema(new Field("responses", + new ArrayOf(FETCH_RESPONSE_TOPIC_V0))); + public static final Schema FETCH_RESPONSE_V1 = new Schema(new Field("throttle_time_ms", + INT32, + "Duration in milliseconds for which the request was throttled" + + " due to quota violation. (Zero if the request did not violate any quota.)", + 0), + new Field("responses", + new ArrayOf(FETCH_RESPONSE_TOPIC_V0))); - public static Schema[] LIST_OFFSET_REQUEST = new Schema[] { LIST_OFFSET_REQUEST_V0 }; - public static Schema[] LIST_OFFSET_RESPONSE = new Schema[] { LIST_OFFSET_RESPONSE_V0 }; + public static final Schema[] FETCH_REQUEST = new Schema[] {FETCH_REQUEST_V0, FETCH_REQUEST_V1}; + public static final Schema[] FETCH_RESPONSE = new Schema[] {FETCH_RESPONSE_V0, FETCH_RESPONSE_V1}; - /* Fetch api */ - public static 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 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 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 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)); + /* Consumer metadata api */ + public static final Schema CONSUMER_METADATA_REQUEST_V0 = new Schema(new Field("group_id", + STRING, + "The consumer group id.")); - public static 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 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 Schema FETCH_RESPONSE_V0 = new Schema(new Field("responses", - new ArrayOf(FETCH_RESPONSE_TOPIC_V0))); + 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}; - public static Schema[] FETCH_REQUEST = new Schema[] { FETCH_REQUEST_V0 }; - public static Schema[] FETCH_RESPONSE = new Schema[] { FETCH_RESPONSE_V0 }; + /* Controlled shutdown api */ + public static final Schema CONTROLLED_SHUTDOWN_REQUEST_V1 = new Schema(new Field("broker_id", + INT32, + "The id of the broker for which controlled shutdown has been requested.")); - /* Consumer metadata api */ - public static Schema CONSUMER_METADATA_REQUEST_V0 = new Schema(new Field("group_id", - STRING, - "The consumer group id.")); + public static final Schema CONTROLLED_SHUTDOWN_PARTITION_V1 = new Schema(new Field("topic", STRING), + new Field("partition", + INT32, + "Topic partition id.")); - public static 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 CONTROLLED_SHUTDOWN_RESPONSE_V1 = new Schema(new Field("error_code", INT16), + new Field("partitions_remaining", + new ArrayOf(CONTROLLED_SHUTDOWN_PARTITION_V1), + "The partitions that the broker still leads.")); - public static Schema[] CONSUMER_METADATA_REQUEST = new Schema[] { CONSUMER_METADATA_REQUEST_V0 }; - public static Schema[] CONSUMER_METADATA_RESPONSE = new Schema[] { CONSUMER_METADATA_RESPONSE_V0 }; + /* V0 is not supported as it would require changes to the request header not to include `clientId` */ + public static final Schema[] CONTROLLED_SHUTDOWN_REQUEST = new Schema[] {null, CONTROLLED_SHUTDOWN_REQUEST_V1}; + public static final Schema[] CONTROLLED_SHUTDOWN_RESPONSE = new Schema[] {null, CONTROLLED_SHUTDOWN_RESPONSE_V1}; /* Join group api */ - public static 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 Schema JOIN_GROUP_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), - new Field("partitions", new ArrayOf(INT32))); - public static 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_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 Schema[] JOIN_GROUP_REQUEST = new Schema[] { JOIN_GROUP_REQUEST_V0 }; - public static Schema[] JOIN_GROUP_RESPONSE = new Schema[] { JOIN_GROUP_RESPONSE_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 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_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}; + + /* Leader and ISR api */ + public static final Schema LEADER_AND_ISR_REQUEST_PARTITION_STATE_V0 = + new Schema(new Field("topic", STRING, "Topic name."), + new Field("partition", INT32, "Topic partition id."), + new Field("controller_epoch", INT32, "The controller epoch."), + new Field("leader", INT32, "The broker id for the leader."), + new Field("leader_epoch", INT32, "The leader epoch."), + new Field("isr", new ArrayOf(INT32), "The in sync replica ids."), + new Field("zk_version", INT32, "The ZK version."), + new Field("replicas", new ArrayOf(INT32), "The replica ids.")); + + public static final Schema LEADER_AND_ISR_REQUEST_LIVE_LEADER_V0 = + new Schema(new Field("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 LEADER_AND_ISR_REQUEST_V0 = new Schema(new Field("controller_id", INT32, "The controller id."), + new Field("controller_epoch", INT32, "The controller epoch."), + new Field("partition_states", + new ArrayOf(LEADER_AND_ISR_REQUEST_PARTITION_STATE_V0)), + new Field("live_leaders", new ArrayOf(LEADER_AND_ISR_REQUEST_LIVE_LEADER_V0))); + + public static final Schema LEADER_AND_ISR_RESPONSE_PARTITION_V0 = new Schema(new Field("topic", STRING, "Topic name."), + new Field("partition", INT32, "Topic partition id."), + new Field("error_code", INT16, "Error code.")); + + public static final Schema LEADER_AND_ISR_RESPONSE_V0 = new Schema(new Field("error_code", INT16, "Error code."), + new Field("partitions", + new ArrayOf(LEADER_AND_ISR_RESPONSE_PARTITION_V0))); - public static Schema HEARTBEAT_RESPONSE_V0 = new Schema(new Field("error_code", - INT16)); + public static final Schema[] LEADER_AND_ISR_REQUEST = new Schema[] {LEADER_AND_ISR_REQUEST_V0}; + public static final Schema[] LEADER_AND_ISR_RESPONSE = new Schema[] {LEADER_AND_ISR_RESPONSE_V0}; - public static Schema[] HEARTBEAT_REQUEST = new Schema[] {HEARTBEAT_REQUEST_V0}; - public static Schema[] HEARTBEAT_RESPONSE = new Schema[] {HEARTBEAT_RESPONSE_V0}; + /* Replica api */ + public static final Schema STOP_REPLICA_REQUEST_PARTITION_V0 = new Schema(new Field("topic", STRING, "Topic name."), + new Field("partition", INT32, "Topic partition id.")); - /* 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 STOP_REPLICA_REQUEST_V0 = new Schema(new Field("controller_id", INT32, "The controller id."), + new Field("controller_epoch", INT32, "The controller epoch."), + new Field("delete_partitions", + INT8, + "Boolean which indicates if replica's partitions must be deleted."), + new Field("partitions", + new ArrayOf(STOP_REPLICA_REQUEST_PARTITION_V0))); + + public static final Schema STOP_REPLICA_RESPONSE_PARTITION_V0 = new Schema(new Field("topic", STRING, "Topic name."), + new Field("partition", INT32, "Topic partition id."), + new Field("error_code", INT16, "Error code.")); + + public static final Schema STOP_REPLICA_RESPONSE_V0 = new Schema(new Field("error_code", INT16, "Error code."), + new Field("partitions", + new ArrayOf(STOP_REPLICA_RESPONSE_PARTITION_V0))); + + public static final Schema[] STOP_REPLICA_REQUEST = new Schema[] {STOP_REPLICA_REQUEST_V0}; + public static final Schema[] STOP_REPLICA_RESPONSE = new Schema[] {STOP_REPLICA_RESPONSE_V0}; + + /* Update metadata api */ + + public static final Schema UPDATE_METADATA_REQUEST_PARTITION_STATE_V0 = LEADER_AND_ISR_REQUEST_PARTITION_STATE_V0; + + public static final Schema UPDATE_METADATA_REQUEST_BROKER_V0 = + new Schema(new Field("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 UPDATE_METADATA_REQUEST_V0 = new Schema(new Field("controller_id", INT32, "The controller id."), + new Field("controller_epoch", INT32, "The controller epoch."), + new Field("partition_states", + new ArrayOf(UPDATE_METADATA_REQUEST_PARTITION_STATE_V0)), + new Field("live_brokers", + new ArrayOf(UPDATE_METADATA_REQUEST_BROKER_V0))); + + public static final Schema UPDATE_METADATA_RESPONSE_V0 = new Schema(new Field("error_code", INT16, "Error code.")); + + public static final Schema UPDATE_METADATA_REQUEST_PARTITION_STATE_V1 = UPDATE_METADATA_REQUEST_PARTITION_STATE_V0; + + public static final Schema UPDATE_METADATA_REQUEST_END_POINT_V1 = + // for some reason, V1 sends `port` before `host` while V0 sends `host` before `port + new Schema(new Field("port", INT32, "The port on which the broker accepts requests."), + new Field("host", STRING, "The hostname of the broker."), + new Field("security_protocol_type", INT16, "The security protocol type.")); + + public static final Schema UPDATE_METADATA_REQUEST_BROKER_V1 = + new Schema(new Field("id", INT32, "The broker id."), + new Field("end_points", new ArrayOf(UPDATE_METADATA_REQUEST_END_POINT_V1))); + + public static final Schema UPDATE_METADATA_REQUEST_V1 = new Schema(new Field("controller_id", INT32, "The controller id."), + new Field("controller_epoch", INT32, "The controller epoch."), + new Field("partition_states", + new ArrayOf(UPDATE_METADATA_REQUEST_PARTITION_STATE_V1)), + new Field("live_brokers", + new ArrayOf(UPDATE_METADATA_REQUEST_BROKER_V1))); + + public static final Schema UPDATE_METADATA_RESPONSE_V1 = UPDATE_METADATA_RESPONSE_V0; + + public static final Schema[] UPDATE_METADATA_REQUEST = new Schema[] {UPDATE_METADATA_REQUEST_V0, UPDATE_METADATA_REQUEST_V1}; + public static final Schema[] UPDATE_METADATA_RESPONSE = new Schema[] {UPDATE_METADATA_RESPONSE_V0, UPDATE_METADATA_RESPONSE_V1}; + + /* an array of all requests and responses with all schema versions; a null value in the inner array means that the + * particular version is not supported */ + 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] = 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.LEADER_AND_ISR.id] = LEADER_AND_ISR_REQUEST; + REQUESTS[ApiKeys.STOP_REPLICA.id] = STOP_REPLICA_REQUEST; + REQUESTS[ApiKeys.UPDATE_METADATA_KEY.id] = UPDATE_METADATA_REQUEST; + REQUESTS[ApiKeys.CONTROLLED_SHUTDOWN_KEY.id] = CONTROLLED_SHUTDOWN_REQUEST; 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] = 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.LEADER_AND_ISR.id] = LEADER_AND_ISR_RESPONSE; + RESPONSES[ApiKeys.STOP_REPLICA.id] = STOP_REPLICA_RESPONSE; + RESPONSES[ApiKeys.UPDATE_METADATA_KEY.id] = UPDATE_METADATA_RESPONSE; + RESPONSES[ApiKeys.CONTROLLED_SHUTDOWN_KEY.id] = CONTROLLED_SHUTDOWN_RESPONSE; RESPONSES[ApiKeys.OFFSET_COMMIT.id] = OFFSET_COMMIT_RESPONSE; RESPONSES[ApiKeys.OFFSET_FETCH.id] = OFFSET_FETCH_RESPONSE; RESPONSES[ApiKeys.CONSUMER_METADATA.id] = CONSUMER_METADATA_RESPONSE; @@ -401,11 +604,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..a624741bff35e --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.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.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"), + /** SSL channel */ + SSL(1, "SSL"), + /** 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 7164701258666..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 @@ -24,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(); 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 121e880a941fc..ef2525e507c8c 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); @@ -124,14 +122,6 @@ public Long getLong(String name) { return (Long) get(name); } - public ByteBuffer getBytes(Field field) { - return (ByteBuffer) get(field); - } - - public ByteBuffer getBytes(String name) { - return (ByteBuffer) get(name); - } - public Object[] getArray(Field field) { return (Object[]) get(field); } @@ -148,11 +138,26 @@ public String getString(String name) { return (String) get(name); } + public ByteBuffer getBytes(Field field) { + Object result = get(field); + if (result instanceof byte[]) + return ByteBuffer.wrap((byte[]) result); + return (ByteBuffer) result; + } + + public ByteBuffer getBytes(String name) { + Object result = get(name); + if (result instanceof byte[]) + return ByteBuffer.wrap((byte[]) result); + return (ByteBuffer) result; + } + /** * 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); @@ -165,6 +170,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); @@ -175,12 +181,13 @@ public Struct set(String name, Object value) { } /** - * 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. + * 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); @@ -190,7 +197,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()); } } @@ -199,6 +206,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)); @@ -227,16 +235,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); @@ -249,7 +261,7 @@ public void validate() { public ByteBuffer[] toBytes() { ByteBuffer buffer = ByteBuffer.allocate(sizeOf()); writeTo(buffer); - return new ByteBuffer[] { buffer }; + return new ByteBuffer[] {buffer}; } @Override 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 f0d5a8286380d..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 @@ -25,14 +25,32 @@ */ 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/ByteBufferOutputStream.java b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java index c7bd2f8852bd9..1c9fbaa958423 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java +++ b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java @@ -24,7 +24,7 @@ */ public class ByteBufferOutputStream extends OutputStream { - private static float REALLOCATION_FACTOR = 1.1f; + private static final float REALLOCATION_FACTOR = 1.1f; private ByteBuffer buffer; 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 5227b2d7ab803..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,7 +20,7 @@ * The compression type to use */ public enum CompressionType { - NONE(0, "none", 1.0f), GZIP(1, "gzip", 0.5f), SNAPPY(2, "snappy", 0.5f), LZ4(3, "lz4", 0.5f), LZ4HC(4, "lz4hc", 0.5f); + 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; @@ -42,8 +42,6 @@ public static CompressionType forId(int id) { return SNAPPY; case 3: return LZ4; - case 4: - return LZ4HC; default: throw new IllegalArgumentException("Unknown compression type id: " + id); } @@ -58,8 +56,6 @@ else if (SNAPPY.name.equals(name)) return SNAPPY; else if (LZ4.name.equals(name)) return LZ4; - else if (LZ4HC.name.equals(name)) - return LZ4HC; 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 index 0323f5f7032dc..e570b29d5ffba 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/Compressor.java +++ b/clients/src/main/java/org/apache/kafka/common/record/Compressor.java @@ -34,16 +34,15 @@ public class Compressor { static private final float COMPRESSION_RATE_ESTIMATION_FACTOR = 1.05f; static private final int COMPRESSION_DEFAULT_BUFFER_SIZE = 1024; - private static float[] typeToRate; - private static int MAX_TYPE_ID = -1; + 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()) { - MAX_TYPE_ID = Math.max(MAX_TYPE_ID, type.id); - } - typeToRate = new float[MAX_TYPE_ID+1]; - for (CompressionType type : CompressionType.values()) { - typeToRate[type.id] = type.rate; + TYPE_TO_RATE[type.id] = type.rate; } } @@ -118,7 +117,7 @@ public void close() { // update the compression ratio float compressionRate = (float) buffer.position() / this.writtenUncompressed; - typeToRate[type.id] = typeToRate[type.id] * COMPRESSION_RATE_DAMPING_FACTOR + + TYPE_TO_RATE[type.id] = TYPE_TO_RATE[type.id] * COMPRESSION_RATE_DAMPING_FACTOR + compressionRate * (1 - COMPRESSION_RATE_DAMPING_FACTOR); } } @@ -192,7 +191,7 @@ public long estimatedBytesWritten() { return bufferStream.buffer().position(); } else { // estimate the written bytes to the underlying byte buffer based on uncompressed written bytes - return (long) (writtenUncompressed * typeToRate[type.id] * COMPRESSION_RATE_ESTIMATION_FACTOR); + return (long) (writtenUncompressed * TYPE_TO_RATE[type.id] * COMPRESSION_RATE_ESTIMATION_FACTOR); } } @@ -209,8 +208,8 @@ static public DataOutputStream wrapForOutput(ByteBufferOutputStream buffer, Comp // dynamically load the snappy class to avoid runtime dependency // on snappy if we are not using it try { - Class SnappyOutputStream = Class.forName("org.xerial.snappy.SnappyOutputStream"); - OutputStream stream = (OutputStream) SnappyOutputStream.getConstructor(OutputStream.class, Integer.TYPE) + 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) { @@ -218,27 +217,13 @@ static public DataOutputStream wrapForOutput(ByteBufferOutputStream buffer, Comp } case LZ4: try { - Class LZ4BlockOutputStream = Class.forName("net.jpountz.lz4.LZ4BlockOutputStream"); - OutputStream stream = (OutputStream) LZ4BlockOutputStream.getConstructor(OutputStream.class) + 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); } - case LZ4HC: - try { - Class factoryClass = Class.forName("net.jpountz.lz4.LZ4Factory"); - Class compressorClass = Class.forName("net.jpountz.lz4.LZ4Compressor"); - Class lz4BlockOutputStream = Class.forName("net.jpountz.lz4.LZ4BlockOutputStream"); - Object factory = factoryClass.getMethod("fastestInstance").invoke(null); - Object compressor = factoryClass.getMethod("highCompressor").invoke(factory); - OutputStream stream = (OutputStream) lz4BlockOutputStream - .getConstructor(OutputStream.class, Integer.TYPE, compressorClass) - .newInstance(buffer, 1 << 16, compressor); - return new DataOutputStream(stream); - } catch (Exception e) { - throw new KafkaException(e); - } default: throw new IllegalArgumentException("Unknown compression type: " + type); } @@ -258,18 +243,17 @@ static public DataInputStream wrapForInput(ByteBufferInputStream buffer, Compres // dynamically load the snappy class to avoid runtime dependency // on snappy if we are not using it try { - Class SnappyInputStream = Class.forName("org.xerial.snappy.SnappyInputStream"); - InputStream stream = (InputStream) SnappyInputStream.getConstructor(InputStream.class) + 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: - case LZ4HC: // dynamically load LZ4 class to avoid runtime dependency try { - Class inputStreamClass = Class.forName("net.jpountz.lz4.LZ4BlockInputStream"); + Class inputStreamClass = Class.forName("org.apache.kafka.common.record.KafkaLZ4BlockInputStream"); InputStream stream = (InputStream) inputStreamClass.getConstructor(InputStream.class) .newInstance(buffer); return new DataInputStream(stream); 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 040e5b91005ed..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 @@ -55,7 +55,7 @@ public static MemoryRecords emptyRecords(ByteBuffer buffer, CompressionType type return emptyRecords(buffer, type, buffer.capacity()); } - public static MemoryRecords iterableRecords(ByteBuffer buffer) { + public static MemoryRecords readableRecords(ByteBuffer buffer) { return new MemoryRecords(buffer, CompressionType.NONE, false, buffer.capacity()); } @@ -94,31 +94,32 @@ public void append(long offset, byte[] key, byte[] value) { * 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. + * 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.writable && - this.capacity >= this.compressor.estimatedBytesWritten() + Records.LOG_OVERHEAD + Record.recordSize(key, value) && - this.sizeLimit >= this.compressor.estimatedBytesWritten(); + 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(); + return !this.writable || this.capacity <= this.compressor.estimatedBytesWritten() || + this.sizeLimit <= this.compressor.estimatedBytesWritten(); } /** * Close this batch for no more appends */ public void close() { - compressor.close(); - writable = false; - buffer = compressor.buffer(); + if (writable) { + compressor.close(); + writable = false; + buffer = compressor.buffer(); + } } /** Write the records in this set to the given channel */ @@ -132,7 +133,7 @@ public int writeTo(GatheringByteChannel channel) throws IOException { public int sizeInBytes() { return compressor.buffer().position(); } - + /** * The compression rate of this record set */ @@ -157,11 +158,40 @@ 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() { - ByteBuffer copy = (ByteBuffer) this.buffer.duplicate().flip(); + 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(); + } public static class RecordsIterator extends AbstractIterator { private final ByteBuffer buffer; @@ -174,7 +204,7 @@ public RecordsIterator(ByteBuffer buffer, CompressionType type, boolean shallow) this.type = type; this.buffer = buffer; this.shallow = shallow; - stream = Compressor.wrapForInput(new ByteBufferInputStream(this.buffer), type); + this.stream = Compressor.wrapForInput(new ByteBufferInputStream(this.buffer), type); } /* @@ -199,15 +229,17 @@ protected LogEntry makeNext() { ByteBuffer rec; if (type == CompressionType.NONE) { rec = buffer.slice(); - buffer.position(buffer.position() + size); + int newPos = buffer.position() + size; + if (newPos > buffer.limit()) + return allDone(); + buffer.position(newPos); rec.limit(size); } else { byte[] recordBuffer = new byte[size]; - stream.read(recordBuffer, 0, size); + stream.readFully(recordBuffer, 0, size); rec = ByteBuffer.wrap(recordBuffer); } LogEntry entry = new LogEntry(offset, new Record(rec)); - entry.record().ensureValid(); // decide whether to go shallow or deep iteration if it is compressed CompressionType compression = entry.record().compressionType(); @@ -215,7 +247,9 @@ protected LogEntry makeNext() { 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 + // 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(); @@ -231,7 +265,7 @@ protected LogEntry makeNext() { } private boolean innerDone() { - return (innerIter == null || !innerIter.hasNext()); + 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 10df9fd8d3f4e..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 @@ -317,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..a696e800358e4 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.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.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); + case STOP_REPLICA: + return StopReplicaRequest.parse(buffer, versionId); + case CONTROLLED_SHUTDOWN_KEY: + return ControlledShutdownRequest.parse(buffer, versionId); + case UPDATE_METADATA_KEY: + return UpdateMetadataRequest.parse(buffer, versionId); + case LEADER_AND_ISR: + return LeaderAndIsrRequest.parse(buffer, versionId); + default: + return null; + } + } +} 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 index 99b52c23d639d..5b3e04a54436b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java @@ -12,21 +12,24 @@ */ 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 AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.CONSUMER_METADATA.id); - private static String GROUP_ID_KEY_NAME = "group_id"; +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(curSchema)); + super(new Struct(CURRENT_SCHEMA)); struct.set(GROUP_ID_KEY_NAME, groupId); this.groupId = groupId; @@ -37,11 +40,26 @@ public ConsumerMetadataRequest(Struct 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) curSchema.read(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 index 8b8f591c4b280..0c250c389a445 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java @@ -21,20 +21,21 @@ import java.nio.ByteBuffer; public class ConsumerMetadataResponse extends AbstractRequestResponse { - private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.CONSUMER_METADATA.id); - private static String ERROR_CODE_KEY_NAME = "error_code"; - private static String COORDINATOR_KEY_NAME = "coordinator"; + + 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 String NODE_ID_KEY_NAME = "node_id"; - private static String HOST_KEY_NAME = "host"; - private static String PORT_KEY_NAME = "port"; + 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(curSchema)); + 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()); @@ -64,6 +65,6 @@ public Node node() { } public static ConsumerMetadataResponse parse(ByteBuffer buffer) { - return new ConsumerMetadataResponse(((Struct) curSchema.read(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/ControlledShutdownRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java new file mode 100644 index 0000000000000..57f51d87db590 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.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.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.Collections; + +public class ControlledShutdownRequest extends AbstractRequest { + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id); + + private static final String BROKER_ID_KEY_NAME = "broker_id"; + + private int brokerId; + + public ControlledShutdownRequest(int brokerId) { + super(new Struct(CURRENT_SCHEMA)); + struct.set(BROKER_ID_KEY_NAME, brokerId); + this.brokerId = brokerId; + } + + public ControlledShutdownRequest(Struct struct) { + super(struct); + brokerId = struct.getInt(BROKER_ID_KEY_NAME); + } + + @Override + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { + switch (versionId) { + case 0: + throw new IllegalArgumentException(String.format("Version 0 is not supported. It is only supported by " + + "the Scala request class for controlled shutdown")); + case 1: + return new ControlledShutdownResponse(Errors.forException(e).code(), Collections.emptySet()); + 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.CONTROLLED_SHUTDOWN_KEY.id))); + } + } + + public int brokerId() { + return brokerId; + } + + public static ControlledShutdownRequest parse(ByteBuffer buffer, int versionId) { + return new ControlledShutdownRequest(ProtoUtils.parseRequest(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id, versionId, buffer)); + } + + public static ControlledShutdownRequest parse(ByteBuffer buffer) { + return new ControlledShutdownRequest((Struct) CURRENT_SCHEMA.read(buffer)); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java new file mode 100644 index 0000000000000..15d600d791994 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java @@ -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 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 java.nio.ByteBuffer; +import java.util.*; + +public class ControlledShutdownResponse extends AbstractRequestResponse { + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id); + + private static final String ERROR_CODE_KEY_NAME = "error_code"; + private static final String PARTITIONS_REMAINING_KEY_NAME = "partitions_remaining"; + + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITION_KEY_NAME = "partition"; + + /** + * Possible error codes: + * + * UNKNOWN(-1) (this is because IllegalStateException may be thrown in `KafkaController.shutdownBroker`, it would be good to improve this) + * BROKER_NOT_AVAILABLE(8) + * STALE_CONTROLLER_EPOCH(11) + */ + private final short errorCode; + + private final Set partitionsRemaining; + + public ControlledShutdownResponse(short errorCode, Set partitionsRemaining) { + super(new Struct(CURRENT_SCHEMA)); + + struct.set(ERROR_CODE_KEY_NAME, errorCode); + + List partitionsRemainingList = new ArrayList<>(partitionsRemaining.size()); + for (TopicPartition topicPartition : partitionsRemaining) { + Struct topicPartitionStruct = struct.instance(PARTITIONS_REMAINING_KEY_NAME); + topicPartitionStruct.set(TOPIC_KEY_NAME, topicPartition.topic()); + topicPartitionStruct.set(PARTITION_KEY_NAME, topicPartition.partition()); + } + struct.set(PARTITIONS_REMAINING_KEY_NAME, partitionsRemainingList.toArray()); + + this.errorCode = errorCode; + this.partitionsRemaining = partitionsRemaining; + } + + public ControlledShutdownResponse(Struct struct) { + super(struct); + errorCode = struct.getShort(ERROR_CODE_KEY_NAME); + Set partitions = new HashSet<>(); + for (Object topicPartitionObj : struct.getArray(PARTITIONS_REMAINING_KEY_NAME)) { + Struct topicPartition = (Struct) topicPartitionObj; + String topic = topicPartition.getString(TOPIC_KEY_NAME); + int partition = topicPartition.getInt(PARTITION_KEY_NAME); + partitions.add(new TopicPartition(topic, partition)); + } + partitionsRemaining = partitions; + } + + public short errorCode() { + return errorCode; + } + + public Set partitionsRemaining() { + return partitionsRemaining; + } + + public static ControlledShutdownResponse parse(ByteBuffer buffer) { + return new ControlledShutdownResponse((Struct) CURRENT_SCHEMA.read(buffer)); + } + + public static ControlledShutdownResponse parse(ByteBuffer buffer, int version) { + return new ControlledShutdownResponse(ProtoUtils.parseResponse(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id, version, buffer)); + } + +} 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 index 2fc471f64f435..feb4109bf8b95 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java @@ -1,49 +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. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES 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; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class FetchRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.FETCH.id); - private static String REPLICA_ID_KEY_NAME = "replica_id"; - private static String MAX_WAIT_KEY_NAME = "max_wait_time"; - private static String MIN_BYTES_KEY_NAME = "min_bytes"; - private static String TOPICS_KEY_NAME = "topics"; +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 String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partitions"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partitions"; // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String FETCH_OFFSET_KEY_NAME = "fetch_offset"; - private static String MAX_BYTES_KEY_NAME = "max_bytes"; + 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; @@ -60,15 +59,25 @@ public PartitionData(long offset, int 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(curSchema)); + 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()) { + for (Map.Entry> topicEntry : topicsData.entrySet()) { Struct topicData = struct.instance(TOPICS_KEY_NAME); topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); List partitionArray = new ArrayList(); @@ -110,6 +119,26 @@ public FetchRequest(Struct struct) { } } + @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, 0); + 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; } @@ -126,7 +155,11 @@ 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) curSchema.read(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 index f719010119951..7b7841579c78d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java @@ -29,21 +29,44 @@ import java.util.List; import java.util.Map; +/** + * This wrapper supports both v0 and v1 of FetchResponse. + */ public class FetchResponse extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.FETCH.id); - private static String RESPONSES_KEY_NAME = "responses"; + + 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 String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partition_responses"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partition_responses"; + private static final String THROTTLE_TIME_KEY_NAME = "throttle_time_ms"; // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String ERROR_CODE_KEY_NAME = "error_code"; - private static String HIGH_WATERMARK_KEY_NAME = "high_watermark"; - private static String RECORD_SET_KEY_NAME = "record_set"; + private static final String PARTITION_KEY_NAME = "partition"; + private static final String ERROR_CODE_KEY_NAME = "error_code"; + + // Default throttle time + private static final int DEFAULT_THROTTLE_TIME = 0; + + /** + * 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; + private final int throttleTime; public static final class PartitionData { public final short errorCode; @@ -57,8 +80,50 @@ public PartitionData(short errorCode, long highWatermark, ByteBuffer recordSet) } } + /** + * Constructor for Version 0 + * @param responseData fetched data grouped by topic-partition + */ public FetchResponse(Map responseData) { - super(new Struct(curSchema)); + super(new Struct(ProtoUtils.responseSchema(ApiKeys.FETCH.id, 0))); + initCommonFields(responseData); + this.responseData = responseData; + this.throttleTime = DEFAULT_THROTTLE_TIME; + } + + /** + * Constructor for Version 1 + * @param responseData fetched data grouped by topic-partition + * @param throttleTime Time in milliseconds the response was throttled + */ + public FetchResponse(Map responseData, int throttleTime) { + super(new Struct(CURRENT_SCHEMA)); + initCommonFields(responseData); + struct.set(THROTTLE_TIME_KEY_NAME, throttleTime); + this.responseData = responseData; + this.throttleTime = throttleTime; + } + + 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); + } + } + this.throttleTime = struct.hasField(THROTTLE_TIME_KEY_NAME) ? struct.getInt(THROTTLE_TIME_KEY_NAME) : DEFAULT_THROTTLE_TIME; + } + + private void initCommonFields(Map responseData) { Map> topicsData = CollectionUtils.groupDataByTopic(responseData); List topicArray = new ArrayList(); @@ -79,32 +144,22 @@ public FetchResponse(Map responseData) { 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 int getThrottleTime() { + return this.throttleTime; + } + public static FetchResponse parse(ByteBuffer buffer) { - return new FetchResponse(((Struct) curSchema.read(buffer))); + return new FetchResponse((Struct) CURRENT_SCHEMA.read(buffer)); + } + + public static FetchResponse parse(ByteBuffer buffer, int version) { + return new FetchResponse((Struct) ProtoUtils.responseSchema(ApiKeys.FETCH.id, version).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 index 9512db2365d8d..89719f13e2870 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java @@ -13,24 +13,26 @@ 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 AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.HEARTBEAT.id); - private static String GROUP_ID_KEY_NAME = "group_id"; - private static String GROUP_GENERATION_ID_KEY_NAME = "group_generation_id"; - private static String CONSUMER_ID_KEY_NAME = "consumer_id"; +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(curSchema)); + 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); @@ -46,6 +48,17 @@ public HeartbeatRequest(Struct struct) { 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; } @@ -58,7 +71,11 @@ 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) curSchema.read(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 index 8997ffc44c18b..96e6ab07dfa77 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java @@ -20,12 +20,22 @@ import java.nio.ByteBuffer; public class HeartbeatResponse extends AbstractRequestResponse { - private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.HEARTBEAT.id); - private static String ERROR_CODE_KEY_NAME = "error_code"; + + 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(curSchema)); + super(new Struct(CURRENT_SCHEMA)); struct.set(ERROR_CODE_KEY_NAME, errorCode); this.errorCode = errorCode; } @@ -40,6 +50,6 @@ public short errorCode() { } public static HeartbeatResponse parse(ByteBuffer buffer) { - return new HeartbeatResponse(((Struct) curSchema.read(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 index d6e91f3931798..1ffe0760b40c4 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java @@ -12,22 +12,28 @@ */ 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 AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.JOIN_GROUP.id); - private static String GROUP_ID_KEY_NAME = "group_id"; - private static String SESSION_TIMEOUT_KEY_NAME = "session_timeout"; - private static String TOPICS_KEY_NAME = "topics"; - private static String CONSUMER_ID_KEY_NAME = "consumer_id"; - private static String STRATEGY_KEY_NAME = "partition_assignment_strategy"; +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; @@ -36,7 +42,7 @@ public class JoinGroupRequest extends AbstractRequestResponse { private final String strategy; public JoinGroupRequest(String groupId, int sessionTimeout, List topics, String consumerId, String strategy) { - super(new Struct(curSchema)); + 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()); @@ -61,6 +67,21 @@ public JoinGroupRequest(Struct struct) { 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; } @@ -81,7 +102,11 @@ 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) curSchema.read(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 index efe89796a7bd0..7bf544ef17068 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java @@ -23,16 +23,29 @@ import java.util.*; public class JoinGroupResponse extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.JOIN_GROUP.id); - private static String ERROR_CODE_KEY_NAME = "error_code"; - private static String GENERATION_ID_KEY_NAME = "group_generation_id"; - private static String CONSUMER_ID_KEY_NAME = "consumer_id"; - private static String ASSIGNED_PARTITIONS_KEY_NAME = "assigned_partitions"; - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partitions"; - - public static int UNKNOWN_GENERATION_ID = -1; - public static String UNKNOWN_CONSUMER_ID = ""; + + 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; @@ -40,7 +53,7 @@ public class JoinGroupResponse extends AbstractRequestResponse { private final List assignedPartitions; public JoinGroupResponse(short errorCode, int generationId, String consumerId, List assignedPartitions) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> partitionsByTopic = CollectionUtils.groupDataByTopic(assignedPartitions); @@ -62,10 +75,6 @@ public JoinGroupResponse(short errorCode, int generationId, String consumerId, L this.assignedPartitions = assignedPartitions; } - public JoinGroupResponse(short errorCode) { - this(errorCode, UNKNOWN_GENERATION_ID, UNKNOWN_CONSUMER_ID, Collections.emptyList()); - } - public JoinGroupResponse(Struct struct) { super(struct); assignedPartitions = new ArrayList(); @@ -97,6 +106,6 @@ public List assignedPartitions() { } public static JoinGroupResponse parse(ByteBuffer buffer) { - return new JoinGroupResponse(((Struct) curSchema.read(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/LeaderAndIsrRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java new file mode 100644 index 0000000000000..002beef4dd2db --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java @@ -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 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.*; + +public class LeaderAndIsrRequest extends AbstractRequest { + + public static class PartitionState { + public final int controllerEpoch; + public final int leader; + public final int leaderEpoch; + public final List isr; + public final int zkVersion; + public final Set replicas; + + public PartitionState(int controllerEpoch, int leader, int leaderEpoch, List isr, int zkVersion, Set replicas) { + this.controllerEpoch = controllerEpoch; + this.leader = leader; + this.leaderEpoch = leaderEpoch; + this.isr = isr; + this.zkVersion = zkVersion; + this.replicas = replicas; + } + + } + + public static final class EndPoint { + public final int id; + public final String host; + public final int port; + + public EndPoint(int id, String host, int port) { + this.id = id; + this.host = host; + this.port = port; + } + } + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.LEADER_AND_ISR.id); + + private static final String CONTROLLER_ID_KEY_NAME = "controller_id"; + private static final String CONTROLLER_EPOCH_KEY_NAME = "controller_epoch"; + private static final String PARTITION_STATES_KEY_NAME = "partition_states"; + private static final String LIVE_LEADERS_KEY_NAME = "live_leaders"; + + // partition_states key names + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITION_KEY_NAME = "partition"; + private static final String LEADER_KEY_NAME = "leader"; + private static final String LEADER_EPOCH_KEY_NAME = "leader_epoch"; + private static final String ISR_KEY_NAME = "isr"; + private static final String ZK_VERSION_KEY_NAME = "zk_version"; + private static final String REPLICAS_KEY_NAME = "replicas"; + + // live_leaders key names + private static final String END_POINT_ID_KEY_NAME = "id"; + private static final String HOST_KEY_NAME = "host"; + private static final String PORT_KEY_NAME = "port"; + + private final int controllerId; + private final int controllerEpoch; + private final Map partitionStates; + private final Set liveLeaders; + + public LeaderAndIsrRequest(int controllerId, int controllerEpoch, Map partitionStates, + Set liveLeaders) { + super(new Struct(CURRENT_SCHEMA)); + struct.set(CONTROLLER_ID_KEY_NAME, controllerId); + struct.set(CONTROLLER_EPOCH_KEY_NAME, controllerEpoch); + + List partitionStatesData = new ArrayList<>(partitionStates.size()); + for (Map.Entry entry : partitionStates.entrySet()) { + Struct partitionStateData = struct.instance(PARTITION_STATES_KEY_NAME); + TopicPartition topicPartition = entry.getKey(); + partitionStateData.set(TOPIC_KEY_NAME, topicPartition.topic()); + partitionStateData.set(PARTITION_KEY_NAME, topicPartition.partition()); + PartitionState partitionState = entry.getValue(); + partitionStateData.set(CONTROLLER_EPOCH_KEY_NAME, partitionState.controllerEpoch); + partitionStateData.set(LEADER_KEY_NAME, partitionState.leader); + partitionStateData.set(LEADER_EPOCH_KEY_NAME, partitionState.leaderEpoch); + partitionStateData.set(ISR_KEY_NAME, partitionState.isr.toArray()); + partitionStateData.set(ZK_VERSION_KEY_NAME, partitionState.zkVersion); + partitionStateData.set(REPLICAS_KEY_NAME, partitionState.replicas.toArray()); + partitionStatesData.add(partitionStateData); + } + struct.set(PARTITION_STATES_KEY_NAME, partitionStatesData.toArray()); + + List leadersData = new ArrayList<>(liveLeaders.size()); + for (EndPoint leader : liveLeaders) { + Struct leaderData = struct.instance(LIVE_LEADERS_KEY_NAME); + leaderData.set(END_POINT_ID_KEY_NAME, leader.id); + leaderData.set(HOST_KEY_NAME, leader.host); + leaderData.set(PORT_KEY_NAME, leader.port); + leadersData.add(leaderData); + } + struct.set(LIVE_LEADERS_KEY_NAME, leadersData.toArray()); + + this.controllerId = controllerId; + this.controllerEpoch = controllerEpoch; + this.partitionStates = partitionStates; + this.liveLeaders = liveLeaders; + } + + public LeaderAndIsrRequest(Struct struct) { + super(struct); + + Map partitionStates = new HashMap<>(); + for (Object partitionStateDataObj : struct.getArray(PARTITION_STATES_KEY_NAME)) { + Struct partitionStateData = (Struct) partitionStateDataObj; + String topic = partitionStateData.getString(TOPIC_KEY_NAME); + int partition = partitionStateData.getInt(PARTITION_KEY_NAME); + int controllerEpoch = partitionStateData.getInt(CONTROLLER_EPOCH_KEY_NAME); + int leader = partitionStateData.getInt(LEADER_KEY_NAME); + int leaderEpoch = partitionStateData.getInt(LEADER_EPOCH_KEY_NAME); + + Object[] isrArray = partitionStateData.getArray(ISR_KEY_NAME); + List isr = new ArrayList<>(isrArray.length); + for (Object r : isrArray) + isr.add((Integer) r); + + int zkVersion = partitionStateData.getInt(ZK_VERSION_KEY_NAME); + + Object[] replicasArray = partitionStateData.getArray(REPLICAS_KEY_NAME); + Set replicas = new HashSet<>(replicasArray.length); + for (Object r : replicasArray) + replicas.add((Integer) r); + + PartitionState partitionState = new PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas); + partitionStates.put(new TopicPartition(topic, partition), partitionState); + + } + + Set leaders = new HashSet<>(); + for (Object leadersDataObj : struct.getArray(LIVE_LEADERS_KEY_NAME)) { + Struct leadersData = (Struct) leadersDataObj; + int id = leadersData.getInt(END_POINT_ID_KEY_NAME); + String host = leadersData.getString(HOST_KEY_NAME); + int port = leadersData.getInt(PORT_KEY_NAME); + leaders.add(new EndPoint(id, host, port)); + } + + controllerId = struct.getInt(CONTROLLER_ID_KEY_NAME); + controllerEpoch = struct.getInt(CONTROLLER_EPOCH_KEY_NAME); + this.partitionStates = partitionStates; + this.liveLeaders = leaders; + } + + @Override + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { + Map responses = new HashMap<>(partitionStates.size()); + for (TopicPartition partition : partitionStates.keySet()) { + responses.put(partition, Errors.forException(e).code()); + } + + switch (versionId) { + case 0: + return new LeaderAndIsrResponse(Errors.NONE.code(), responses); + 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.LEADER_AND_ISR.id))); + } + } + + public int controllerId() { + return controllerId; + } + + public int controllerEpoch() { + return controllerEpoch; + } + + public Map partitionStates() { + return partitionStates; + } + + public Set liveLeaders() { + return liveLeaders; + } + + public static LeaderAndIsrRequest parse(ByteBuffer buffer, int versionId) { + return new LeaderAndIsrRequest(ProtoUtils.parseRequest(ApiKeys.LEADER_AND_ISR.id, versionId, buffer)); + } + + public static LeaderAndIsrRequest parse(ByteBuffer buffer) { + return new LeaderAndIsrRequest((Struct) CURRENT_SCHEMA.read(buffer)); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java new file mode 100644 index 0000000000000..3a6f4ee43a896 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java @@ -0,0 +1,105 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES 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.HashMap; +import java.util.List; +import java.util.Map; + +public class LeaderAndIsrResponse extends AbstractRequestResponse { + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.LEADER_AND_ISR.id); + + private static final String ERROR_CODE_KEY_NAME = "error_code"; + private static final String PARTITIONS_KEY_NAME = "partitions"; + + private static final String PARTITIONS_TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_PARTITION_KEY_NAME = "partition"; + private static final String PARTITIONS_ERROR_CODE_KEY_NAME = "error_code"; + + /** + * Possible error code: + * + * STALE_CONTROLLER_EPOCH (11) + */ + private final short errorCode; + + private final Map responses; + + public LeaderAndIsrResponse(Map responses) { + this(Errors.NONE.code(), responses); + } + + public LeaderAndIsrResponse(short errorCode, Map responses) { + super(new Struct(CURRENT_SCHEMA)); + + struct.set(ERROR_CODE_KEY_NAME, errorCode); + + List responseDatas = new ArrayList<>(responses.size()); + for (Map.Entry response : responses.entrySet()) { + Struct partitionData = struct.instance(PARTITIONS_KEY_NAME); + TopicPartition partition = response.getKey(); + partitionData.set(PARTITIONS_TOPIC_KEY_NAME, partition.topic()); + partitionData.set(PARTITIONS_PARTITION_KEY_NAME, partition.partition()); + partitionData.set(PARTITIONS_ERROR_CODE_KEY_NAME, response.getValue()); + responseDatas.add(partitionData); + } + + struct.set(PARTITIONS_KEY_NAME, responseDatas.toArray()); + struct.set(ERROR_CODE_KEY_NAME, errorCode); + + this.responses = responses; + this.errorCode = errorCode; + } + + public LeaderAndIsrResponse(Struct struct) { + super(struct); + + responses = new HashMap<>(); + for (Object responseDataObj : struct.getArray(PARTITIONS_KEY_NAME)) { + Struct responseData = (Struct) responseDataObj; + String topic = responseData.getString(PARTITIONS_TOPIC_KEY_NAME); + int partition = responseData.getInt(PARTITIONS_PARTITION_KEY_NAME); + short errorCode = responseData.getShort(PARTITIONS_ERROR_CODE_KEY_NAME); + responses.put(new TopicPartition(topic, partition), errorCode); + } + + errorCode = struct.getShort(ERROR_CODE_KEY_NAME); + } + + public Map responses() { + return responses; + } + + public short errorCode() { + return errorCode; + } + + public static LeaderAndIsrResponse parse(ByteBuffer buffer, int version) { + return new LeaderAndIsrResponse(ProtoUtils.parseResponse(ApiKeys.LEADER_AND_ISR.id, version, buffer)); + } + + public static LeaderAndIsrResponse parse(ByteBuffer buffer) { + return new LeaderAndIsrResponse((Struct) CURRENT_SCHEMA.read(buffer)); + } + +} 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 index 99364c1ca464f..8dfd811a3ef00 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java @@ -18,6 +18,7 @@ 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; @@ -29,19 +30,23 @@ import java.util.List; import java.util.Map; -public class ListOffsetRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.LIST_OFFSETS.id); - private static String REPLICA_ID_KEY_NAME = "replica_id"; - private static String TOPICS_KEY_NAME = "topics"; +public class ListOffsetRequest extends AbstractRequest { + + public static final long EARLIEST_TIMESTAMP = -2L; + public static final long LATEST_TIMESTAMP = -1L; + + 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 String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partitions"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partitions"; // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String TIMESTAMP_KEY_NAME = "timestamp"; - private static String MAX_NUM_OFFSETS_KEY_NAME = "max_num_offsets"; + 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; @@ -55,9 +60,13 @@ public PartitionData(long timestamp, int maxNumOffsets) { this.maxNumOffsets = maxNumOffsets; } } + + public ListOffsetRequest(Map offsetData) { + this(-1, offsetData); + } public ListOffsetRequest(int replicaId, Map offsetData) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> topicsData = CollectionUtils.groupDataByTopic(offsetData); struct.set(REPLICA_ID_KEY_NAME, replicaId); @@ -100,6 +109,24 @@ public ListOffsetRequest(Struct struct) { } } + @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; } @@ -108,7 +135,11 @@ 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) curSchema.read(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 index ac239712f1184..f70608600fc4e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java @@ -30,17 +30,27 @@ import java.util.Map; public class ListOffsetResponse extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.LIST_OFFSETS.id); - private static String RESPONSES_KEY_NAME = "responses"; + + 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 String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partition_responses"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partition_responses"; // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String ERROR_CODE_KEY_NAME = "error_code"; - private static String OFFSETS_KEY_NAME = "offsets"; + 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; @@ -55,7 +65,7 @@ public PartitionData(short errorCode, List offsets) { } public ListOffsetResponse(Map responseData) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> topicsData = CollectionUtils.groupDataByTopic(responseData); List topicArray = new ArrayList(); @@ -103,6 +113,6 @@ public Map responseData() { } public static ListOffsetResponse parse(ByteBuffer buffer) { - return new ListOffsetResponse(((Struct) curSchema.read(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 index b22ca1dce65f6..f70e8dac70c1a 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java @@ -14,21 +14,28 @@ 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 AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id); - private static String TOPICS_KEY_NAME = "topics"; +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(curSchema)); + super(new Struct(CURRENT_SCHEMA)); struct.set(TOPICS_KEY_NAME, topics.toArray()); this.topics = topics; } @@ -42,11 +49,32 @@ public MetadataRequest(Struct struct) { } } + @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) curSchema.read(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 index d97962d384017..c8f2d087dfc85 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java @@ -28,35 +28,54 @@ import org.apache.kafka.common.protocol.types.Struct; public class MetadataResponse extends AbstractRequestResponse { - private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.METADATA.id); - private static String BROKERS_KEY_NAME = "brokers"; - private static String TOPIC_METATDATA_KEY_NAME = "topic_metadata"; + + 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 String NODE_ID_KEY_NAME = "node_id"; - private static String HOST_KEY_NAME = "host"; - private static String PORT_KEY_NAME = "port"; + 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 String TOPIC_ERROR_CODE_KEY_NAME = "topic_error_code"; - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITION_METADATA_KEY_NAME = "partition_metadata"; + 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 String PARTITION_ERROR_CODE_KEY_NAME = "partition_error_code"; - private static String PARTITION_KEY_NAME = "partition_id"; - private static String LEADER_KEY_NAME = "leader"; - private static String REPLICAS_KEY_NAME = "replicas"; - private static String ISR_KEY_NAME = "isr"; + 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; - public MetadataResponse(Cluster cluster) { - super(new Struct(curSchema)); + /** + * 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()) { + 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()); @@ -66,27 +85,33 @@ public MetadataResponse(Cluster cluster) { struct.set(BROKERS_KEY_NAME, brokerArray.toArray()); List topicArray = new ArrayList(); - for (String topic: cluster.topics()) { + for (String topic : cluster.topics()) { Struct topicData = struct.instance(TOPIC_METATDATA_KEY_NAME); - topicData.set(TOPIC_ERROR_CODE_KEY_NAME, (short)0); // no error + topicData.set(TOPIC_KEY_NAME, topic); - 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, (short)0); // no error - 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); + 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()); } - topicData.set(PARTITION_METADATA_KEY_NAME, partitionArray.toArray()); + topicArray.add(topicData); } struct.set(TOPIC_METATDATA_KEY_NAME, topicArray.toArray()); @@ -147,6 +172,6 @@ public Cluster cluster() { } public static MetadataResponse parse(ByteBuffer buffer) { - return new MetadataResponse(((Struct) curSchema.read(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 index 3ee5cbad55ce8..03df1e780c071 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java @@ -3,9 +3,9 @@ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the * License. You may obtain a copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the * specific language governing permissions and limitations under the License. @@ -14,6 +14,7 @@ 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; @@ -28,41 +29,60 @@ /** * This wrapper supports both v0 and v1 of OffsetCommitRequest. */ -public class OffsetCommitRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_COMMIT.id); - private static String GROUP_ID_KEY_NAME = "group_id"; - private static String GENERATION_ID_KEY_NAME = "group_generation_id"; - private static String CONSUMER_ID_KEY_NAME = "consumer_id"; - private static String TOPICS_KEY_NAME = "topics"; +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 String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partitions"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partitions"; // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String COMMIT_OFFSET_KEY_NAME = "offset"; - private static String TIMESTAMP_KEY_NAME = "timestamp"; - private static String METADATA_KEY_NAME = "metadata"; + 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 int generationId; 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 long timestamp; 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); + } } /** @@ -73,10 +93,12 @@ public PartitionData(long offset, long timestamp, String metadata) { @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; } @@ -87,8 +109,9 @@ public OffsetCommitRequest(String groupId, Map of * @param consumerId * @param offsetData */ + @Deprecated public OffsetCommitRequest(String groupId, int generationId, String consumerId, Map offsetData) { - super(new Struct(curSchema)); + super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, 1))); initCommonFields(groupId, offsetData); struct.set(GENERATION_ID_KEY_NAME, generationId); @@ -96,6 +119,29 @@ public OffsetCommitRequest(String groupId, int generationId, String 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; } @@ -104,6 +150,7 @@ private void initCommonFields(String groupId, Map 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()); @@ -113,7 +160,9 @@ private void initCommonFields(String groupId, Map Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset); - partitionData.set(TIMESTAMP_KEY_NAME, fetchPartitionData.timestamp); + // 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); } @@ -125,20 +174,7 @@ private void initCommonFields(String groupId, Map public OffsetCommitRequest(Struct struct) { super(struct); - 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 offset = partitionResponse.getLong(COMMIT_OFFSET_KEY_NAME); - long timestamp = partitionResponse.getLong(TIMESTAMP_KEY_NAME); - String metadata = partitionResponse.getString(METADATA_KEY_NAME); - PartitionData partitionData = new PartitionData(offset, timestamp, metadata); - offsetData.put(new TopicPartition(topic, partition), partitionData); - } - } + groupId = struct.getString(GROUP_ID_KEY_NAME); // This field only exists in v1. if (struct.hasField(GENERATION_ID_KEY_NAME)) @@ -151,6 +187,52 @@ public OffsetCommitRequest(Struct struct) { 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() { @@ -165,16 +247,20 @@ 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))); + return new OffsetCommitRequest((Struct) schema.read(buffer)); } public static OffsetCommitRequest parse(ByteBuffer buffer) { - return new OffsetCommitRequest(((Struct) curSchema.read(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 index 711232ac61378..a1633330dc0c9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java @@ -26,21 +26,34 @@ import java.util.Map; public class OffsetCommitResponse extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_COMMIT.id); - private static String RESPONSES_KEY_NAME = "responses"; + + 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 String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partition_responses"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partition_responses"; // partition level fields - private static String PARTITION_KEY_NAME = "partition"; - private static String ERROR_CODE_KEY_NAME = "error_code"; + 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(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> topicsData = CollectionUtils.groupDataByTopic(responseData); @@ -82,6 +95,6 @@ public Map responseData() { } public static OffsetCommitResponse parse(ByteBuffer buffer) { - return new OffsetCommitResponse(((Struct) curSchema.read(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 index 90d5135b97a44..6ee75973d6484 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java @@ -14,6 +14,7 @@ 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; @@ -21,32 +22,31 @@ 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 AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_FETCH.id); - private static String GROUP_ID_KEY_NAME = "group_id"; - private static String TOPICS_KEY_NAME = "topics"; +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 String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partitions"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partitions"; // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - - public static final int DEFAULT_GENERATION_ID = -1; - public static final String DEFAULT_CONSUMER_ID = ""; + 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(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> topicsData = CollectionUtils.groupDataByTopic(partitions); @@ -82,7 +82,28 @@ public OffsetFetchRequest(Struct struct) { } } 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; @@ -92,7 +113,11 @@ 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) curSchema.read(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 index 6b7c269ad7679..3dc8521296e37 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java @@ -3,43 +3,58 @@ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the * License. You may obtain a copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES 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; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - public class OffsetFetchResponse extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_FETCH.id); - private static String RESPONSES_KEY_NAME = "responses"; + + 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 String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partition_responses"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partition_responses"; // partition level fields - private static String PARTITION_KEY_NAME = "partition"; - private static String COMMIT_OFFSET_KEY_NAME = "offset"; - private static String METADATA_KEY_NAME = "metadata"; - private static String ERROR_CODE_KEY_NAME = "error_code"; + 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"; - private final Map responseData; + 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; @@ -51,15 +66,19 @@ public PartitionData(long offset, String metadata, short errorCode) { this.metadata = metadata; this.errorCode = errorCode; } + + public boolean hasError() { + return this.errorCode != Errors.NONE.code(); + } } public OffsetFetchResponse(Map responseData) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> topicsData = CollectionUtils.groupDataByTopic(responseData); List topicArray = new ArrayList(); - for (Map.Entry> entries: topicsData.entrySet()) { + for (Map.Entry> entries : topicsData.entrySet()) { Struct topicData = struct.instance(RESPONSES_KEY_NAME); topicData.set(TOPIC_KEY_NAME, entries.getKey()); List partitionArray = new ArrayList(); @@ -102,6 +121,6 @@ public Map responseData() { } public static OffsetFetchResponse parse(ByteBuffer buffer) { - return new OffsetFetchResponse(((Struct) curSchema.read(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 index 3dbba8a360f11..5663f2cf7e0b7 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -15,6 +15,7 @@ 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; @@ -26,26 +27,27 @@ import java.util.List; import java.util.Map; -public class ProduceRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.PRODUCE.id); - private static String ACKS_KEY_NAME = "acks"; - private static String TIMEOUT_KEY_NAME = "timeout"; - private static String TOPIC_DATA_KEY_NAME = "topic_data"; +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 String TOPIC_KEY_NAME = "topic"; - private static String PARTITION_DATA_KEY_NAME = "data"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITION_DATA_KEY_NAME = "data"; // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String RECORD_SET_KEY_NAME = "record_set"; + 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(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> recordsByTopic = CollectionUtils.groupDataByTopic(partitionRecords); struct.set(ACKS_KEY_NAME, acks); struct.set(TIMEOUT_KEY_NAME, timeout); @@ -87,6 +89,27 @@ public ProduceRequest(Struct struct) { 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, 0); + 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; } @@ -99,7 +122,11 @@ 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) curSchema.read(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 index 5220464913e6e..286855018972c 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -25,23 +25,80 @@ import java.util.List; import java.util.Map; +/** + * This wrapper supports both v0 and v1 of ProduceResponse. + */ public class ProduceResponse extends AbstractRequestResponse { - private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id); - private static String RESPONSES_KEY_NAME = "responses"; + + 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 String TOPIC_KEY_NAME = "topic"; - private static String PARTITION_RESPONSES_KEY_NAME = "partition_responses"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITION_RESPONSES_KEY_NAME = "partition_responses"; + private static final String THROTTLE_TIME_KEY_NAME = "throttle_time_ms"; // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String ERROR_CODE_KEY_NAME = "error_code"; - private static String BASE_OFFSET_KEY_NAME = "base_offset"; + 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; + private static final int DEFAULT_THROTTLE_TIME = 0; + + /** + * Possible error code: + * + * TODO + */ + + private static final String BASE_OFFSET_KEY_NAME = "base_offset"; private final Map responses; + private final int throttleTime; + /** + * Constructor for Version 0 + * @param responses Produced data grouped by topic-partition + */ public ProduceResponse(Map responses) { - super(new Struct(curSchema)); + super(new Struct(ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, 0))); + initCommonFields(responses); + this.responses = responses; + this.throttleTime = DEFAULT_THROTTLE_TIME; + } + + /** + * Constructor for Version 1 + * @param responses Produced data grouped by topic-partition + * @param throttleTime Time in milliseconds the response was throttled + */ + public ProduceResponse(Map responses, int throttleTime) { + super(new Struct(CURRENT_SCHEMA)); + initCommonFields(responses); + struct.set(THROTTLE_TIME_KEY_NAME, throttleTime); + this.responses = responses; + this.throttleTime = throttleTime; + } + + public ProduceResponse(Struct struct) { + super(struct); + responses = new HashMap(); + for (Object topicResponse : struct.getArray(RESPONSES_KEY_NAME)) { + Struct topicRespStruct = (Struct) topicResponse; + String topic = topicRespStruct.getString(TOPIC_KEY_NAME); + for (Object partResponse : topicRespStruct.getArray(PARTITION_RESPONSES_KEY_NAME)) { + Struct partRespStruct = (Struct) partResponse; + int partition = partRespStruct.getInt(PARTITION_KEY_NAME); + short errorCode = partRespStruct.getShort(ERROR_CODE_KEY_NAME); + long offset = partRespStruct.getLong(BASE_OFFSET_KEY_NAME); + TopicPartition tp = new TopicPartition(topic, partition); + responses.put(tp, new PartitionResponse(errorCode, offset)); + } + } + this.throttleTime = struct.getInt(THROTTLE_TIME_KEY_NAME); + } + + private void initCommonFields(Map responses) { Map> responseByTopic = CollectionUtils.groupDataByTopic(responses); List topicDatas = new ArrayList(responseByTopic.size()); for (Map.Entry> entry : responseByTopic.entrySet()) { @@ -50,40 +107,25 @@ public ProduceResponse(Map responses) { 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); + 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 int getThrottleTime() { + return this.throttleTime; + } + public static final class PartitionResponse { public short errorCode; public long baseOffset; @@ -107,6 +149,6 @@ public String toString() { } public static ProduceResponse parse(ByteBuffer buffer) { - return new ProduceResponse(((Struct) curSchema.read(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 f459a2a62f7b9..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 @@ -26,10 +26,10 @@ */ 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 short apiKey; private final short apiVersion; 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 27cbf390c7f14..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 @@ -25,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; 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 dd63853e15f50..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 @@ -30,7 +30,7 @@ */ 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 int correlationId; @@ -50,7 +50,7 @@ public int 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/requests/StopReplicaRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java new file mode 100644 index 0000000000000..85ac39421d8c9 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java @@ -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 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.*; + +public class StopReplicaRequest extends AbstractRequest { + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.STOP_REPLICA.id); + + private static final String CONTROLLER_ID_KEY_NAME = "controller_id"; + private static final String CONTROLLER_EPOCH_KEY_NAME = "controller_epoch"; + private static final String DELETE_PARTITIONS_KEY_NAME = "delete_partitions"; + private static final String PARTITIONS_KEY_NAME = "partitions"; + + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITION_KEY_NAME = "partition"; + + private final int controllerId; + private final int controllerEpoch; + private final boolean deletePartitions; + private final Set partitions; + + public StopReplicaRequest(int controllerId, int controllerEpoch, boolean deletePartitions, Set partitions) { + super(new Struct(CURRENT_SCHEMA)); + + struct.set(CONTROLLER_ID_KEY_NAME, controllerId); + struct.set(CONTROLLER_EPOCH_KEY_NAME, controllerEpoch); + struct.set(DELETE_PARTITIONS_KEY_NAME, deletePartitions ? (byte) 1 : (byte) 0); + + List partitionDatas = new ArrayList<>(partitions.size()); + for (TopicPartition partition : partitions) { + Struct partitionData = struct.instance(PARTITIONS_KEY_NAME); + partitionData.set(TOPIC_KEY_NAME, partition.topic()); + partitionData.set(PARTITION_KEY_NAME, partition.partition()); + partitionDatas.add(partitionData); + } + + struct.set(PARTITIONS_KEY_NAME, partitionDatas.toArray()); + + this.controllerId = controllerId; + this.controllerEpoch = controllerEpoch; + this.deletePartitions = deletePartitions; + this.partitions = partitions; + } + + public StopReplicaRequest(Struct struct) { + super(struct); + + partitions = new HashSet<>(); + for (Object partitionDataObj : struct.getArray(PARTITIONS_KEY_NAME)) { + Struct partitionData = (Struct) partitionDataObj; + String topic = partitionData.getString(TOPIC_KEY_NAME); + int partition = partitionData.getInt(PARTITION_KEY_NAME); + partitions.add(new TopicPartition(topic, partition)); + } + + controllerId = struct.getInt(CONTROLLER_ID_KEY_NAME); + controllerEpoch = struct.getInt(CONTROLLER_EPOCH_KEY_NAME); + deletePartitions = ((byte) struct.get(DELETE_PARTITIONS_KEY_NAME)) != 0; + } + + @Override + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { + Map responses = new HashMap<>(partitions.size()); + for (TopicPartition partition : partitions) { + responses.put(partition, Errors.forException(e).code()); + } + + switch (versionId) { + case 0: + return new StopReplicaResponse(Errors.NONE.code(), responses); + 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.STOP_REPLICA.id))); + } + } + + public int controllerId() { + return controllerId; + } + + public int controllerEpoch() { + return controllerEpoch; + } + + public boolean deletePartitions() { + return deletePartitions; + } + + public Set partitions() { + return partitions; + } + + public static StopReplicaRequest parse(ByteBuffer buffer, int versionId) { + return new StopReplicaRequest(ProtoUtils.parseRequest(ApiKeys.STOP_REPLICA.id, versionId, buffer)); + } + + public static StopReplicaRequest parse(ByteBuffer buffer) { + return new StopReplicaRequest((Struct) CURRENT_SCHEMA.read(buffer)); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java new file mode 100644 index 0000000000000..4fa1cac0891a2 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java @@ -0,0 +1,101 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES 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.*; + +public class StopReplicaResponse extends AbstractRequestResponse { + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.STOP_REPLICA.id); + + private static final String ERROR_CODE_KEY_NAME = "error_code"; + private static final String PARTITIONS_KEY_NAME = "partitions"; + + private static final String PARTITIONS_TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_PARTITION_KEY_NAME = "partition"; + private static final String PARTITIONS_ERROR_CODE_KEY_NAME = "error_code"; + + private final Map responses; + private final short errorCode; + + /** + * Possible error code: + * + * STALE_CONTROLLER_EPOCH (11) + */ + + public StopReplicaResponse(Map responses) { + this(Errors.NONE.code(), responses); + } + + public StopReplicaResponse(short errorCode, Map responses) { + super(new Struct(CURRENT_SCHEMA)); + + struct.set(ERROR_CODE_KEY_NAME, errorCode); + + List responseDatas = new ArrayList<>(responses.size()); + for (Map.Entry response : responses.entrySet()) { + Struct partitionData = struct.instance(PARTITIONS_KEY_NAME); + TopicPartition partition = response.getKey(); + partitionData.set(PARTITIONS_TOPIC_KEY_NAME, partition.topic()); + partitionData.set(PARTITIONS_PARTITION_KEY_NAME, partition.partition()); + partitionData.set(PARTITIONS_ERROR_CODE_KEY_NAME, response.getValue()); + responseDatas.add(partitionData); + } + + struct.set(PARTITIONS_KEY_NAME, responseDatas.toArray()); + struct.set(ERROR_CODE_KEY_NAME, errorCode); + + this.responses = responses; + this.errorCode = errorCode; + } + + public StopReplicaResponse(Struct struct) { + super(struct); + + responses = new HashMap<>(); + for (Object responseDataObj : struct.getArray(PARTITIONS_KEY_NAME)) { + Struct responseData = (Struct) responseDataObj; + String topic = responseData.getString(PARTITIONS_TOPIC_KEY_NAME); + int partition = responseData.getInt(PARTITIONS_PARTITION_KEY_NAME); + short errorCode = responseData.getShort(PARTITIONS_ERROR_CODE_KEY_NAME); + responses.put(new TopicPartition(topic, partition), errorCode); + } + + errorCode = struct.getShort(ERROR_CODE_KEY_NAME); + } + + public Map responses() { + return responses; + } + + public short errorCode() { + return errorCode; + } + + public static StopReplicaResponse parse(ByteBuffer buffer, int versionId) { + return new StopReplicaResponse(ProtoUtils.parseRequest(ApiKeys.STOP_REPLICA.id, versionId, buffer)); + } + + public static StopReplicaResponse parse(ByteBuffer buffer) { + return new StopReplicaResponse((Struct) CURRENT_SCHEMA.read(buffer)); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java new file mode 100644 index 0000000000000..f9f76be7dd01e --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java @@ -0,0 +1,291 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES 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.SecurityProtocol; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.Struct; + +import java.nio.ByteBuffer; +import java.util.*; + +public class UpdateMetadataRequest extends AbstractRequest { + + public static final class PartitionState { + public final int controllerEpoch; + public final int leader; + public final int leaderEpoch; + public final List isr; + public final int zkVersion; + public final Set replicas; + + public PartitionState(int controllerEpoch, int leader, int leaderEpoch, List isr, int zkVersion, Set replicas) { + this.controllerEpoch = controllerEpoch; + this.leader = leader; + this.leaderEpoch = leaderEpoch; + this.isr = isr; + this.zkVersion = zkVersion; + this.replicas = replicas; + } + + } + + public static final class Broker { + public final int id; + public final Map endPoints; + + public Broker(int id, Map endPoints) { + this.id = id; + this.endPoints = endPoints; + } + } + + public static final class EndPoint { + public final String host; + public final int port; + + public EndPoint(String host, int port) { + this.host = host; + this.port = port; + } + } + + @Deprecated + public static final class BrokerEndPoint { + public final int id; + public final String host; + public final int port; + + public BrokerEndPoint(int id, String host, int port) { + this.id = id; + this.host = host; + this.port = port; + } + } + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.UPDATE_METADATA_KEY.id); + + private static final String CONTROLLER_ID_KEY_NAME = "controller_id"; + private static final String CONTROLLER_EPOCH_KEY_NAME = "controller_epoch"; + private static final String PARTITION_STATES_KEY_NAME = "partition_states"; + private static final String LIVE_BROKERS_KEY_NAME = "live_brokers"; + + // PartitionState key names + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITION_KEY_NAME = "partition"; + private static final String LEADER_KEY_NAME = "leader"; + private static final String LEADER_EPOCH_KEY_NAME = "leader_epoch"; + private static final String ISR_KEY_NAME = "isr"; + private static final String ZK_VERSION_KEY_NAME = "zk_version"; + private static final String REPLICAS_KEY_NAME = "replicas"; + + // Broker key names + private static final String BROKER_ID_KEY_NAME = "id"; + private static final String ENDPOINTS_KEY_NAME = "end_points"; + + // EndPoint key names + private static final String HOST_KEY_NAME = "host"; + private static final String PORT_KEY_NAME = "port"; + private static final String SECURITY_PROTOCOL_TYPE_KEY_NAME = "security_protocol_type"; + + private final int controllerId; + private final int controllerEpoch; + private final Map partitionStates; + private final Set liveBrokers; + + /** + * Constructor for version 0. + */ + @Deprecated + public UpdateMetadataRequest(int controllerId, int controllerEpoch, Set liveBrokers, + Map partitionStates) { + this(0, controllerId, controllerEpoch, partitionStates, + brokerEndPointsToBrokers(liveBrokers)); + } + + private static Set brokerEndPointsToBrokers(Set brokerEndPoints) { + Set brokers = new HashSet<>(brokerEndPoints.size()); + for (BrokerEndPoint brokerEndPoint : brokerEndPoints) { + Map endPoints = Collections.singletonMap(SecurityProtocol.PLAINTEXT, + new EndPoint(brokerEndPoint.host, brokerEndPoint.port)); + brokers.add(new Broker(brokerEndPoint.id, endPoints)); + } + return brokers; + } + + /** + * Constructor for version 1. + */ + public UpdateMetadataRequest(int controllerId, int controllerEpoch, Map partitionStates, Set liveBrokers) { + this(1, controllerId, controllerEpoch, partitionStates, liveBrokers); + } + + private UpdateMetadataRequest(int version, int controllerId, int controllerEpoch, Map partitionStates, Set liveBrokers) { + super(new Struct(ProtoUtils.requestSchema(ApiKeys.UPDATE_METADATA_KEY.id, version))); + struct.set(CONTROLLER_ID_KEY_NAME, controllerId); + struct.set(CONTROLLER_EPOCH_KEY_NAME, controllerEpoch); + + List partitionStatesData = new ArrayList<>(partitionStates.size()); + for (Map.Entry entry : partitionStates.entrySet()) { + Struct partitionStateData = struct.instance(PARTITION_STATES_KEY_NAME); + TopicPartition topicPartition = entry.getKey(); + partitionStateData.set(TOPIC_KEY_NAME, topicPartition.topic()); + partitionStateData.set(PARTITION_KEY_NAME, topicPartition.partition()); + PartitionState partitionState = entry.getValue(); + partitionStateData.set(CONTROLLER_EPOCH_KEY_NAME, partitionState.controllerEpoch); + partitionStateData.set(LEADER_KEY_NAME, partitionState.leader); + partitionStateData.set(LEADER_EPOCH_KEY_NAME, partitionState.leaderEpoch); + partitionStateData.set(ISR_KEY_NAME, partitionState.isr.toArray()); + partitionStateData.set(ZK_VERSION_KEY_NAME, partitionState.zkVersion); + partitionStateData.set(REPLICAS_KEY_NAME, partitionState.replicas.toArray()); + partitionStatesData.add(partitionStateData); + } + struct.set(PARTITION_STATES_KEY_NAME, partitionStatesData.toArray()); + + List brokersData = new ArrayList<>(liveBrokers.size()); + for (Broker broker : liveBrokers) { + Struct brokerData = struct.instance(LIVE_BROKERS_KEY_NAME); + brokerData.set(BROKER_ID_KEY_NAME, broker.id); + + if (version == 0) { + EndPoint endPoint = broker.endPoints.get(SecurityProtocol.PLAINTEXT); + brokerData.set(HOST_KEY_NAME, endPoint.host); + brokerData.set(PORT_KEY_NAME, endPoint.port); + } else { + List endPointsData = new ArrayList<>(broker.endPoints.size()); + for (Map.Entry entry : broker.endPoints.entrySet()) { + Struct endPointData = brokerData.instance(ENDPOINTS_KEY_NAME); + endPointData.set(PORT_KEY_NAME, entry.getValue().port); + endPointData.set(HOST_KEY_NAME, entry.getValue().host); + endPointData.set(SECURITY_PROTOCOL_TYPE_KEY_NAME, entry.getKey().id); + endPointsData.add(endPointData); + + } + brokerData.set(ENDPOINTS_KEY_NAME, endPointsData.toArray()); + } + + brokersData.add(brokerData); + } + struct.set(LIVE_BROKERS_KEY_NAME, brokersData.toArray()); + + this.controllerId = controllerId; + this.controllerEpoch = controllerEpoch; + this.partitionStates = partitionStates; + this.liveBrokers = liveBrokers; + } + + public UpdateMetadataRequest(Struct struct) { + super(struct); + + Map partitionStates = new HashMap<>(); + for (Object partitionStateDataObj : struct.getArray(PARTITION_STATES_KEY_NAME)) { + Struct partitionStateData = (Struct) partitionStateDataObj; + String topic = partitionStateData.getString(TOPIC_KEY_NAME); + int partition = partitionStateData.getInt(PARTITION_KEY_NAME); + int controllerEpoch = partitionStateData.getInt(CONTROLLER_EPOCH_KEY_NAME); + int leader = partitionStateData.getInt(LEADER_KEY_NAME); + int leaderEpoch = partitionStateData.getInt(LEADER_EPOCH_KEY_NAME); + + Object[] isrArray = partitionStateData.getArray(ISR_KEY_NAME); + List isr = new ArrayList<>(isrArray.length); + for (Object r : isrArray) + isr.add((Integer) r); + + int zkVersion = partitionStateData.getInt(ZK_VERSION_KEY_NAME); + + Object[] replicasArray = partitionStateData.getArray(REPLICAS_KEY_NAME); + Set replicas = new HashSet<>(replicasArray.length); + for (Object r : replicasArray) + replicas.add((Integer) r); + + PartitionState partitionState = new PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas); + partitionStates.put(new TopicPartition(topic, partition), partitionState); + + } + + Set liveBrokers = new HashSet<>(); + + for (Object brokerDataObj : struct.getArray(LIVE_BROKERS_KEY_NAME)) { + Struct brokerData = (Struct) brokerDataObj; + int brokerId = brokerData.getInt(BROKER_ID_KEY_NAME); + + // V0 + if (brokerData.hasField(HOST_KEY_NAME)) { + String host = brokerData.getString(HOST_KEY_NAME); + int port = brokerData.getInt(PORT_KEY_NAME); + Map endPoints = new HashMap<>(1); + endPoints.put(SecurityProtocol.PLAINTEXT, new EndPoint(host, port)); + liveBrokers.add(new Broker(brokerId, endPoints)); + } else { // V1 + Map endPoints = new HashMap<>(); + for (Object endPointDataObj : brokerData.getArray(ENDPOINTS_KEY_NAME)) { + Struct endPointData = (Struct) endPointDataObj; + int port = endPointData.getInt(PORT_KEY_NAME); + String host = endPointData.getString(HOST_KEY_NAME); + short protocolTypeId = endPointData.getShort(SECURITY_PROTOCOL_TYPE_KEY_NAME); + endPoints.put(SecurityProtocol.forId(protocolTypeId), new EndPoint(host, port)); + } + liveBrokers.add(new Broker(brokerId, endPoints)); + } + + } + + controllerId = struct.getInt(CONTROLLER_ID_KEY_NAME); + controllerEpoch = struct.getInt(CONTROLLER_EPOCH_KEY_NAME); + this.partitionStates = partitionStates; + this.liveBrokers = liveBrokers; + } + + @Override + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { + switch (versionId) { + case 0: + case 1: + return new UpdateMetadataResponse(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.UPDATE_METADATA_KEY.id))); + } + } + + public int controllerId() { + return controllerId; + } + + public int controllerEpoch() { + return controllerEpoch; + } + + public Map partitionStates() { + return partitionStates; + } + + public Set liveBrokers() { + return liveBrokers; + } + + public static UpdateMetadataRequest parse(ByteBuffer buffer, int versionId) { + return new UpdateMetadataRequest(ProtoUtils.parseRequest(ApiKeys.UPDATE_METADATA_KEY.id, versionId, buffer)); + } + + public static UpdateMetadataRequest parse(ByteBuffer buffer) { + return new UpdateMetadataRequest((Struct) CURRENT_SCHEMA.read(buffer)); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java new file mode 100644 index 0000000000000..5bec437c45e62 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.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.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 UpdateMetadataResponse extends AbstractRequestResponse { + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.UPDATE_METADATA_KEY.id); + + private static final String ERROR_CODE_KEY_NAME = "error_code"; + + /** + * Possible error code: + * + * STALE_CONTROLLER_EPOCH (11) + */ + private final short errorCode; + + public UpdateMetadataResponse(short errorCode) { + super(new Struct(CURRENT_SCHEMA)); + struct.set(ERROR_CODE_KEY_NAME, errorCode); + this.errorCode = errorCode; + } + + public UpdateMetadataResponse(Struct struct) { + super(struct); + errorCode = struct.getShort(ERROR_CODE_KEY_NAME); + } + + public short errorCode() { + return errorCode; + } + + public static UpdateMetadataResponse parse(ByteBuffer buffer) { + return new UpdateMetadataResponse((Struct) CURRENT_SCHEMA.read(buffer)); + } + + public static UpdateMetadataResponse parse(ByteBuffer buffer, int version) { + return new UpdateMetadataResponse(ProtoUtils.parseResponse(ApiKeys.UPDATE_METADATA_KEY.id, version, buffer)); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java b/clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java new file mode 100644 index 0000000000000..fbbeb9e8bc7a3 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java @@ -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. + */ + +package org.apache.kafka.common.security.auth; + +import java.util.Map; +import java.security.Principal; + +import org.apache.kafka.common.network.TransportLayer; +import org.apache.kafka.common.network.Authenticator; +import org.apache.kafka.common.KafkaException; + +/** DefaultPrincipalBuilder which return transportLayer's peer Principal **/ + +public class DefaultPrincipalBuilder implements PrincipalBuilder { + + public void configure(Map configs) {} + + public Principal buildPrincipal(TransportLayer transportLayer, Authenticator authenticator) throws KafkaException { + try { + return transportLayer.peerPrincipal(); + } catch (Exception e) { + throw new KafkaException("Failed to build principal due to: ", e); + } + } + + public void close() throws KafkaException {} + +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java b/clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java new file mode 100644 index 0000000000000..06c59d1c08317 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java @@ -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 org.apache.kafka.common.security.auth; + +import java.security.Principal; + +public class KafkaPrincipal implements Principal { + public static final String SEPARATOR = ":"; + public static final String USER_TYPE = "User"; + public final static KafkaPrincipal ANONYMOUS = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "ANONYMOUS"); + + private String principalType; + private String name; + + public KafkaPrincipal(String principalType, String name) { + if (principalType == null || name == null) { + throw new IllegalArgumentException("principalType and name can not be null"); + } + this.principalType = principalType; + this.name = name; + } + + public static KafkaPrincipal fromString(String str) { + if (str == null || str.isEmpty()) { + throw new IllegalArgumentException("expected a string in format principalType:principalName but got " + str); + } + + String[] split = str.split(SEPARATOR, 2); + + if (split == null || split.length != 2) { + throw new IllegalArgumentException("expected a string in format principalType:principalName but got " + str); + } + + return new KafkaPrincipal(split[0], split[1]); + } + + @Override + public String toString() { + return principalType + SEPARATOR + name; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (!(o instanceof KafkaPrincipal)) return false; + + KafkaPrincipal that = (KafkaPrincipal) o; + + if (!principalType.equals(that.principalType)) return false; + return name.equals(that.name); + + } + + @Override + public int hashCode() { + int result = principalType.hashCode(); + result = 31 * result + name.hashCode(); + return result; + } + + @Override + public String getName() { + return name; + } + + public String getPrincipalType() { + return principalType; + } +} + + + diff --git a/clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java b/clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java new file mode 100644 index 0000000000000..b7cc378bde73c --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.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.common.security.auth; + +/* + * PrincipalBuilder for Authenticator + */ + +import org.apache.kafka.common.network.TransportLayer; +import org.apache.kafka.common.network.Authenticator; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.Configurable; + +import java.util.Map; +import java.security.Principal; + +public interface PrincipalBuilder extends Configurable { + + /** + * configure this class with give key-value pair + */ + public void configure(Map configs); + + /** + * Returns Principal + * @param TransportLayer + * @param Authenticator + */ + Principal buildPrincipal(TransportLayer transportLayer, Authenticator authenticator) throws KafkaException; + + /** + * Close this PrincipalBuilder + */ + public void close() throws KafkaException; + +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/ssl/SSLFactory.java b/clients/src/main/java/org/apache/kafka/common/security/ssl/SSLFactory.java new file mode 100644 index 0000000000000..f79b65cdfd014 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/ssl/SSLFactory.java @@ -0,0 +1,210 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.security.ssl; + +import java.util.Map; +import java.util.List; +import java.io.FileInputStream; +import java.io.IOException; +import java.security.GeneralSecurityException; +import java.security.KeyStore; + +import javax.net.ssl.*; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.Configurable; +import org.apache.kafka.common.config.SSLConfigs; + + +public class SSLFactory implements Configurable { + + public enum Mode { CLIENT, SERVER }; + private String protocol; + private String provider; + private String kmfAlgorithm; + private String tmfAlgorithm; + private SecurityStore keystore = null; + private String keyPassword; + private SecurityStore truststore; + private String[] cipherSuites; + private String[] enabledProtocols; + private String endpointIdentification; + private SSLContext sslContext; + private boolean needClientAuth; + private boolean wantClientAuth; + private final Mode mode; + + + public SSLFactory(Mode mode) { + this.mode = mode; + } + + @Override + public void configure(Map configs) throws KafkaException { + this.protocol = (String) configs.get(SSLConfigs.SSL_PROTOCOL_CONFIG); + this.provider = (String) configs.get(SSLConfigs.SSL_PROVIDER_CONFIG); + + if (configs.get(SSLConfigs.SSL_CIPHER_SUITES_CONFIG) != null) { + List cipherSuitesList = (List) configs.get(SSLConfigs.SSL_CIPHER_SUITES_CONFIG); + this.cipherSuites = (String[]) cipherSuitesList.toArray(new String[cipherSuitesList.size()]); + } + + if (configs.get(SSLConfigs.SSL_ENABLED_PROTOCOLS_CONFIG) != null) { + List enabledProtocolsList = (List) configs.get(SSLConfigs.SSL_ENABLED_PROTOCOLS_CONFIG); + this.enabledProtocols = (String[]) enabledProtocolsList.toArray(new String[enabledProtocolsList.size()]); + } + + if (configs.containsKey(SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG)) { + this.endpointIdentification = (String) configs.get(SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG); + } + + if (configs.containsKey(SSLConfigs.SSL_CLIENT_AUTH_CONFIG)) { + String clientAuthConfig = (String) configs.get(SSLConfigs.SSL_CLIENT_AUTH_CONFIG); + if (clientAuthConfig.equals("required")) + this.needClientAuth = true; + else if (clientAuthConfig.equals("requested")) + this.wantClientAuth = true; + } + + this.kmfAlgorithm = (String) configs.get(SSLConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG); + this.tmfAlgorithm = (String) configs.get(SSLConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG); + + if (checkKeyStoreConfigs(configs)) { + createKeystore((String) configs.get(SSLConfigs.SSL_KEYSTORE_TYPE_CONFIG), + (String) configs.get(SSLConfigs.SSL_KEYSTORE_LOCATION_CONFIG), + (String) configs.get(SSLConfigs.SSL_KEYSTORE_PASSWORD_CONFIG), + (String) configs.get(SSLConfigs.SSL_KEY_PASSWORD_CONFIG)); + } + + createTruststore((String) configs.get(SSLConfigs.SSL_TRUSTSTORE_TYPE_CONFIG), + (String) configs.get(SSLConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG), + (String) configs.get(SSLConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG)); + try { + this.sslContext = createSSLContext(); + } catch (Exception e) { + throw new KafkaException(e); + } + } + + + private SSLContext createSSLContext() throws GeneralSecurityException, IOException { + SSLContext sslContext; + if (provider != null) + sslContext = SSLContext.getInstance(protocol, provider); + else + sslContext = SSLContext.getInstance(protocol); + + KeyManager[] keyManagers = null; + if (keystore != null) { + String kmfAlgorithm = this.kmfAlgorithm != null ? this.kmfAlgorithm : KeyManagerFactory.getDefaultAlgorithm(); + KeyManagerFactory kmf = KeyManagerFactory.getInstance(kmfAlgorithm); + KeyStore ks = keystore.load(); + String keyPassword = this.keyPassword != null ? this.keyPassword : keystore.password; + kmf.init(ks, keyPassword.toCharArray()); + keyManagers = kmf.getKeyManagers(); + } + + String tmfAlgorithm = this.tmfAlgorithm != null ? this.tmfAlgorithm : TrustManagerFactory.getDefaultAlgorithm(); + TrustManagerFactory tmf = TrustManagerFactory.getInstance(tmfAlgorithm); + KeyStore ts = truststore == null ? null : truststore.load(); + tmf.init(ts); + + sslContext.init(keyManagers, tmf.getTrustManagers(), null); + return sslContext; + } + + public SSLEngine createSSLEngine(String peerHost, int peerPort) { + SSLEngine sslEngine = sslContext.createSSLEngine(peerHost, peerPort); + if (cipherSuites != null) sslEngine.setEnabledCipherSuites(cipherSuites); + if (enabledProtocols != null) sslEngine.setEnabledProtocols(enabledProtocols); + + if (mode == Mode.SERVER) { + sslEngine.setUseClientMode(false); + if (needClientAuth) + sslEngine.setNeedClientAuth(needClientAuth); + else + sslEngine.setWantClientAuth(wantClientAuth); + } else { + sslEngine.setUseClientMode(true); + SSLParameters sslParams = sslEngine.getSSLParameters(); + sslParams.setEndpointIdentificationAlgorithm(endpointIdentification); + sslEngine.setSSLParameters(sslParams); + } + return sslEngine; + } + + /** + * Returns a configured SSLContext. + * @return SSLContext. + */ + public SSLContext sslContext() { + return sslContext; + } + + private void createKeystore(String type, String path, String password, String keyPassword) { + if (path == null && password != null) { + throw new KafkaException("SSL key store password is not specified."); + } else if (path != null && password == null) { + throw new KafkaException("SSL key store is not specified, but key store password is specified."); + } else if (path != null && password != null) { + this.keystore = new SecurityStore(type, path, password); + this.keyPassword = keyPassword; + } + } + + private void createTruststore(String type, String path, String password) { + if (path == null && password != null) { + throw new KafkaException("SSL key store password is not specified."); + } else if (path != null && password == null) { + throw new KafkaException("SSL key store is not specified, but key store password is specified."); + } else if (path != null && password != null) { + this.truststore = new SecurityStore(type, path, password); + } + } + + private boolean checkKeyStoreConfigs(Map configs) { + return configs.containsKey(SSLConfigs.SSL_KEYSTORE_TYPE_CONFIG) && + configs.containsKey(SSLConfigs.SSL_KEYSTORE_LOCATION_CONFIG) && + configs.containsKey(SSLConfigs.SSL_KEYSTORE_PASSWORD_CONFIG) && + configs.containsKey(SSLConfigs.SSL_KEY_PASSWORD_CONFIG); + } + + private class SecurityStore { + private final String type; + private final String path; + private final String password; + + private SecurityStore(String type, String path, String password) { + this.type = type == null ? KeyStore.getDefaultType() : type; + this.path = path; + this.password = password; + } + + private KeyStore load() throws GeneralSecurityException, IOException { + FileInputStream in = null; + try { + KeyStore ks = KeyStore.getInstance(type); + in = new FileInputStream(path); + ks.load(in, password.toCharArray()); + return ks; + } finally { + if (in != null) in.close(); + } + } + } + +} diff --git a/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala b/clients/src/main/java/org/apache/kafka/common/serialization/ByteArrayDeserializer.java similarity index 58% rename from core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala rename to clients/src/main/java/org/apache/kafka/common/serialization/ByteArrayDeserializer.java index d0f07e0cbbdac..d89b3ff0509eb 100644 --- a/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ByteArrayDeserializer.java @@ -10,19 +10,25 @@ * an "AS IS" BASIS, WITHOUT WARRANTIES 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.common.requests.JoinGroupResponse -import java.nio.ByteBuffer +package org.apache.kafka.common.serialization; -object JoinGroupResponseAndHeader { - def readFrom(buffer: ByteBuffer): JoinGroupResponseAndHeader = { - val correlationId = buffer.getInt - val body = JoinGroupResponse.parse(buffer) - new JoinGroupResponseAndHeader(correlationId, body) - } -} +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; + } -case class JoinGroupResponseAndHeader(override val correlationId: Int, override val body: JoinGroupResponse) - extends GenericResponseAndHeader(correlationId, body, RequestKeys.nameForKey(RequestKeys.JoinGroupKey), None) { + @Override + public void close() { + // nothing to do + } } diff --git a/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala b/clients/src/main/java/org/apache/kafka/common/serialization/ByteArraySerializer.java similarity index 58% rename from core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala rename to clients/src/main/java/org/apache/kafka/common/serialization/ByteArraySerializer.java index 9a71faae3138a..beaef948e38f2 100644 --- a/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ByteArraySerializer.java @@ -10,19 +10,25 @@ * an "AS IS" BASIS, WITHOUT WARRANTIES 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.common.requests.HeartbeatResponse -import java.nio.ByteBuffer +package org.apache.kafka.common.serialization; -object HeartbeatResponseAndHeader { - def readFrom(buffer: ByteBuffer): HeartbeatResponseAndHeader = { - val correlationId = buffer.getInt - val body = HeartbeatResponse.parse(buffer) - new HeartbeatResponseAndHeader(correlationId, body) - } -} +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; + } -case class HeartbeatResponseAndHeader(override val correlationId: Int, override val body: HeartbeatResponse) - extends GenericResponseAndHeader(correlationId, body, RequestKeys.nameForKey(RequestKeys.HeartbeatKey), None) { + @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/LongDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/LongDeserializer.java new file mode 100644 index 0000000000000..37983e4a4c031 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/LongDeserializer.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 LongDeserializer implements Deserializer { + + public void configure(Map configs, boolean isKey) { + // nothing to do + } + + public Long deserialize(String topic, byte[] data) { + if (data == null) + return null; + if (data.length != 8) { + throw new SerializationException("Size of data received by LongDeserializer is " + + "not 8"); + } + + long 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/LongSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/LongSerializer.java new file mode 100644 index 0000000000000..31005299e7a41 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/LongSerializer.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.common.serialization; + +import java.util.Map; + +public class LongSerializer implements Serializer { + + public void configure(Map configs, boolean isKey) { + // nothing to do + } + + public byte[] serialize(String topic, Long data) { + if (data == null) + return null; + + return new byte[] { + (byte) (data >>> 56), + (byte) (data >>> 48), + (byte) (data >>> 40), + (byte) (data >>> 32), + (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/AppInfoParser.java b/clients/src/main/java/org/apache/kafka/common/utils/AppInfoParser.java new file mode 100644 index 0000000000000..d0fd12a7ded71 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/utils/AppInfoParser.java @@ -0,0 +1,97 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.utils; + +import java.lang.management.ManagementFactory; +import java.util.Properties; + +import javax.management.JMException; +import javax.management.MBeanServer; +import javax.management.ObjectName; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class AppInfoParser { + private static final Logger log = LoggerFactory.getLogger(AppInfoParser.class); + private static String version = "unknown"; + private static String commitId = "unknown"; + + static { + try { + Properties props = new Properties(); + props.load(AppInfoParser.class.getResourceAsStream("/kafka/kafka-version.properties")); + version = props.getProperty("version", version).trim(); + commitId = props.getProperty("commitId", commitId).trim(); + } catch (Exception e) { + log.warn("Error while loading kafka-version.properties :" + e.getMessage()); + } + } + + public static String getVersion() { + return version; + } + + public static String getCommitId() { + return commitId; + } + + public static void registerAppInfo(String prefix, String id) { + try { + ObjectName name = new ObjectName(prefix + ":type=app-info,id=" + id); + AppInfo mBean = new AppInfo(); + ManagementFactory.getPlatformMBeanServer().registerMBean(mBean, name); + } catch (JMException e) { + log.warn("Error registering AppInfo mbean", e); + } + } + + public static void unregisterAppInfo(String prefix, String id) { + MBeanServer server = ManagementFactory.getPlatformMBeanServer(); + try { + ObjectName name = new ObjectName(prefix + ":type=app-info,id=" + id); + if (server.isRegistered(name)) + server.unregisterMBean(name); + } catch (JMException e) { + log.warn("Error unregistering AppInfo mbean", e); + } + } + + public interface AppInfoMBean { + public String getVersion(); + public String getCommitId(); + } + + public static class AppInfo implements AppInfoMBean { + + public AppInfo() { + log.info("Kafka version : " + AppInfoParser.getVersion()); + log.info("Kafka commitId : " + AppInfoParser.getCommitId()); + } + + @Override + public String getVersion() { + return AppInfoParser.getVersion(); + } + + @Override + public String getCommitId() { + return AppInfoParser.getCommitId(); + } + + } +} 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 047ca98ef6dd0..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 @@ -30,7 +30,7 @@ public class Crc32 implements Checksum { /** * Compute the CRC32 of the byte array - * + * * @param bytes The array to compute the checksum for * @return The CRC32 */ @@ -40,7 +40,7 @@ public static long crc32(byte[] bytes) { /** * 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 @@ -79,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; @@ -95,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 */ } @@ -118,7 +118,7 @@ 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)]; } /** @@ -131,2075 +131,257 @@ final public void updateInt(int input) { 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/SystemTime.java b/clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java index d682bd46ec382..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 @@ -36,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 a0827f576e8c3..fa7c92f08502c --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -3,31 +3,55 @@ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the * License. You may obtain a copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES 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.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; 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 { - private static final Pattern HOST_PORT_PATTERN = Pattern.compile("\\[?(.+?)\\]?:(\\d+)"); + // 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 String NL = System.getProperty("line.separator"); + 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 - * + * * @param bytes The byte array * @return The string */ @@ -41,7 +65,7 @@ public static String utf8(byte[] bytes) { /** * Turn a string into a utf8 byte[] - * + * * @param string The string * @return The byte[] */ @@ -55,7 +79,7 @@ public static byte[] utf8(String string) { /** * 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 */ @@ -65,7 +89,7 @@ public static long readUnsignedInt(ByteBuffer buffer) { /** * 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 @@ -74,9 +98,37 @@ 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. - * + * * @param buffer The buffer to write to * @param value The value to write */ @@ -86,7 +138,7 @@ public static void writetUnsignedInt(ByteBuffer buffer, long value) { /** * 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 @@ -95,17 +147,61 @@ public static void writeUnsignedInt(ByteBuffer buffer, int index, long value) { buffer.putInt(index, (int) (value & 0xffffffffL)); } + /** + * Write an unsigned integer in little-endian format to the {@link OutputStream}. + * + * @param out The stream to write to + * @param value The value to write + */ + 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); + } + + /** + * Write an unsigned integer in little-endian format to a byte array + * at a given offset. + * + * @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 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; } /** * Get the length for UTF8-encoding a string without encoding it first - * + * * @param s The string to calculate the length for * @return The length when serialized */ @@ -151,7 +247,7 @@ public static byte[] toArray(ByteBuffer buffer, int offset, int size) { /** * Check that the parameter t is not null - * + * * @param t The object to check * @return t if it isn't null * @throws NullPointerException if t is null. @@ -163,19 +259,44 @@ 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 */ - public static Object newInstance(Class c) { + public static T newInstance(Class c) { try { return c.newInstance(); } catch (IllegalAccessException e) { throw new KafkaException("Could not instantiate class " + c.getName(), e); } catch (InstantiationException e) { throw new KafkaException("Could not instantiate class " + c.getName() + " Does it have a public no-argument constructor?", e); + } catch (NullPointerException e) { + throw new KafkaException("Requested class was null", e); } } + /** + * Look up the class by name and instantiate it. + * @param klass class name + * @param base super class of the class to be instantiated + * @param + * @return the new instance + */ + public static T newInstance(String klass, Class base) throws ClassNotFoundException { + return Utils.newInstance(Class.forName(klass).asSubclass(base)); + } + /** * Generates 32 bit murmur2 hash from byte array * @param data byte array to hash @@ -210,7 +331,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; } @@ -253,4 +374,195 @@ public static String formatAddress(String host, Integer port) { ? "[" + 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()); + } + + /** + * Check if the given ByteBuffer capacity + * @param existingBuffer ByteBuffer capacity to check + * @param newLength new length for the ByteBuffer. + * returns ByteBuffer + */ + public static ByteBuffer ensureCapacity(ByteBuffer existingBuffer, int newLength) { + if (newLength > existingBuffer.capacity()) { + ByteBuffer newBuffer = ByteBuffer.allocate(newLength); + existingBuffer.flip(); + newBuffer.put(existingBuffer); + return newBuffer; + } + return existingBuffer; + } + + /* + * Creates a set + * @param elems the elements + * @param the type of element + * @return Set + */ + public static HashSet mkSet(T... elems) { + return new HashSet<>(Arrays.asList(elems)); + } + + /** + * Recursively delete the given file/directory and any subfiles (if any exist) + * + * @param file The root file at which to begin deleting + */ + public static void delete(File file) { + if (file == null) { + return; + } else if (file.isDirectory()) { + File[] files = file.listFiles(); + if (files != null) { + for (File f : files) + delete(f); + } + file.delete(); + } else { + file.delete(); + } + } + + /** + * Returns an empty collection if this list is null + * @param other + * @return + */ + public static List safe(List other) { + return other == null ? Collections.emptyList() : other; + } } diff --git a/clients/src/test/java/org/apache/kafka/common/utils/ClientUtilsTest.java b/clients/src/test/java/org/apache/kafka/clients/ClientUtilsTest.java similarity index 97% rename from clients/src/test/java/org/apache/kafka/common/utils/ClientUtilsTest.java rename to clients/src/test/java/org/apache/kafka/clients/ClientUtilsTest.java index 6e37ea553f73d..d6a4019c4b815 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/ClientUtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/ClientUtilsTest.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.common.utils; +package org.apache.kafka.clients; import org.apache.kafka.common.config.ConfigException; import org.junit.Test; @@ -39,4 +39,4 @@ public void testNoPort() { 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..b7160a1996e8e --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java @@ -0,0 +1,214 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES 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.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; +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 AtomicReference backgroundError = new AtomicReference(); + + @After + public void tearDown() { + assertNull("Exception in background thread : " + backgroundError.get(), 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()); + // Perform metadata update when an update is requested on the async fetch thread + // This simulates the metadata update sequence in KafkaProducer + while (t1.isAlive() || t2.isAlive()) { + if (metadata.timeToNextUpdate(time) == 0) { + metadata.update(TestUtils.singletonCluster(topic, 1), time); + time += refreshBackoffMs; + } + Thread.sleep(1); + } + 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()); + + metadata.needMetadataForAllTopics(true); + metadata.update(null, time); + assertEquals(100, metadata.timeToNextUpdate(1000)); + } + + @Test + public void testUpdateWithNeedMetadataForAllTopics() { + long time = 0; + metadata.update(Cluster.empty(), time); + metadata.needMetadataForAllTopics(true); + + final List expectedTopics = Collections.singletonList("topic"); + metadata.setTopics(expectedTopics); + metadata.update(new Cluster( + Collections.singletonList(new Node(0, "host1", 1000)), + Arrays.asList( + new PartitionInfo("topic", 0, null, null, null), + new PartitionInfo("topic1", 0, null, null, null))), + 100); + + assertArrayEquals("Metadata got updated with wrong set of topics.", + expectedTopics.toArray(), metadata.topics().toArray()); + + metadata.needMetadataForAllTopics(false); + } + + @Test + public void testListenerGetsNotifiedOfUpdate() { + long time = 0; + final Set topics = new HashSet<>(); + metadata.update(Cluster.empty(), time); + metadata.addListener(new Metadata.Listener() { + @Override + public void onMetadataUpdate(Cluster cluster) { + topics.clear(); + topics.addAll(cluster.topics()); + } + }); + + metadata.update(new Cluster( + Arrays.asList(new Node(0, "host1", 1000)), + Arrays.asList( + new PartitionInfo("topic", 0, null, null, null), + new PartitionInfo("topic1", 0, null, null, null))), + 100); + + assertEquals("Listener did not update topics list correctly", + new HashSet<>(Arrays.asList("topic", "topic1")), topics); + } + + @Test + public void testListenerCanUnregister() { + long time = 0; + final Set topics = new HashSet<>(); + metadata.update(Cluster.empty(), time); + final Metadata.Listener listener = new Metadata.Listener() { + @Override + public void onMetadataUpdate(Cluster cluster) { + topics.clear(); + topics.addAll(cluster.topics()); + } + }; + metadata.addListener(listener); + + metadata.update(new Cluster( + Collections.singletonList(new Node(0, "host1", 1000)), + Arrays.asList( + new PartitionInfo("topic", 0, null, null, null), + new PartitionInfo("topic1", 0, null, null, null))), + 100); + + metadata.removeListener(listener); + + metadata.update(new Cluster( + Arrays.asList(new Node(0, "host1", 1000)), + Arrays.asList( + new PartitionInfo("topic2", 0, null, null, null), + new PartitionInfo("topic3", 0, null, null, null))), + 100); + + assertEquals("Listener did not update topics list correctly", + new HashSet<>(Arrays.asList("topic", "topic1")), topics); + } + + + 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(e.toString()); + } + } + } + }; + 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 index aae8d4a1e9827..67d894d4e6af7 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MockClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/MockClient.java @@ -1,3 +1,19 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.kafka.clients; import java.util.ArrayDeque; @@ -18,12 +34,33 @@ * A mock network client for use testing code */ public class MockClient implements KafkaClient { + public static final RequestMatcher ALWAYS_TRUE = new RequestMatcher() { + @Override + public boolean matches(ClientRequest request) { + return true; + } + }; + + private class FutureResponse { + public final Struct responseBody; + public final boolean disconnected; + public final RequestMatcher requestMatcher; + + public FutureResponse(Struct responseBody, boolean disconnected, RequestMatcher requestMatcher) { + this.responseBody = responseBody; + this.disconnected = disconnected; + this.requestMatcher = requestMatcher; + } + + } 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; @@ -36,12 +73,21 @@ public boolean isReady(Node node, long now) { @Override public boolean ready(Node node, long now) { - boolean found = isReady(node, now); ready.add(node.id()); - return found; + return true; + } + + @Override + public long connectionDelay(Node node, long now) { + return 0; } - public void disconnect(Integer node) { + @Override + public boolean connectionFailed(Node node) { + return false; + } + + public void disconnect(String node) { Iterator iter = requests.iterator(); while (iter.hasNext()) { ClientRequest request = iter.next(); @@ -54,10 +100,30 @@ public void disconnect(Integer node) { } @Override - public List poll(List requests, long timeoutMs, long now) { - this.requests.addAll(requests); + public void send(ClientRequest request, long now) { + if (!futureResponses.isEmpty()) { + FutureResponse futureResp = futureResponses.poll(); + if (!futureResp.requestMatcher.matches(request)) + throw new IllegalStateException("Next in line response did not match expected request"); + + ClientResponse resp = new ClientResponse(request, time.milliseconds(), futureResp.disconnected, futureResp.responseBody); + responses.add(resp); + } else { + request.setSendTimeMs(now); + this.requests.add(request); + } + } + + @Override + public List poll(long timeoutMs, long now) { List copy = new ArrayList(this.responses); - this.responses.clear(); + + while (!this.responses.isEmpty()) { + ClientResponse response = this.responses.poll(); + if (response.request().hasCallback()) + response.request().callback().onComplete(response); + } + return copy; } @@ -66,8 +132,45 @@ public Queue 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(), false, body)); + responses.add(new ClientResponse(request, time.milliseconds(), disconnected, body)); + } + + public void prepareResponse(Struct body) { + prepareResponse(ALWAYS_TRUE, body, false); + } + + /** + * Prepare a response for a request matching the provided matcher. If the matcher does not + * match, {@link #send(ClientRequest)} will throw IllegalStateException + * @param matcher The matcher to apply + * @param body The response body + */ + public void prepareResponse(RequestMatcher matcher, Struct body) { + prepareResponse(matcher, body, false); + } + + public void prepareResponse(Struct body, boolean disconnected) { + prepareResponse(ALWAYS_TRUE, body, disconnected); + } + + /** + * Prepare a response for a request matching the provided matcher. If the matcher does not + * match, {@link #send(ClientRequest)} will throw IllegalStateException + * @param matcher The matcher to apply + * @param body The response body + * @param disconnected Whether the request was disconnected + */ + public void prepareResponse(RequestMatcher matcher, Struct body, boolean disconnected) { + futureResponses.add(new FutureResponse(body, disconnected, matcher)); + } + + public void setNode(Node node) { + this.node = node; } @Override @@ -75,11 +178,21 @@ 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 RequestHeader nextRequestHeader(ApiKeys key, short version) { + return new RequestHeader(key.id, version, "mock", correlation++); + } + @Override public void wakeup() { } @@ -88,9 +201,24 @@ public void wakeup() { public void close() { } + @Override + public void close(String nodeId) { + ready.remove(Integer.valueOf(nodeId)); + } + @Override public Node leastLoadedNode(long now) { - return null; + return this.node; + } + + /** + * The RequestMatcher provides a way to match a particular request to a response prepared + * through {@link #prepareResponse(RequestMatcher, Struct)}. Basically this allows testers + * to inspect the request body for the type of the request or for specific fields that should be set, + * and to fail the test if it doesn't match. + */ + public interface RequestMatcher { + boolean matches(ClientRequest request); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index 1a55242e9399f..237989689e074 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -1,3 +1,19 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.kafka.clients; import static org.junit.Assert.assertEquals; @@ -5,12 +21,10 @@ import static org.junit.Assert.assertTrue; import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; -import org.apache.kafka.clients.producer.internals.Metadata; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; @@ -31,52 +45,72 @@ public class NetworkClientTest { + private final int requestTimeoutMs = 1000; 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); + private NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, 0, 64 * 1024, 64 * 1024, requestTimeoutMs); + + private NetworkClient clientWithStaticNodes = new NetworkClient(selector, new ManualMetadataUpdater(Arrays.asList(node)), + "mock-static", Integer.MAX_VALUE, 0, 64 * 1024, 64 * 1024, requestTimeoutMs); @Before public void setup() { metadata.update(cluster, time.milliseconds()); } - @Test - public void testReadyAndDisconnect() { - List reqs = new ArrayList(); - 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(reqs, 1, time.milliseconds()); - selector.clear(); - assertTrue("Now the client is ready", client.ready(node, time.milliseconds())); - selector.disconnect(node.id()); - client.poll(reqs, 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, + 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.poll(Arrays.asList(request), 1, time.milliseconds()); + client.send(request, time.milliseconds()); + client.poll(1, time.milliseconds()); } @Test public void testSimpleRequestResponse() { + checkSimpleRequestResponse(client); + } + + @Test + public void testSimpleRequestResponseWithStaticNodes() { + checkSimpleRequestResponse(clientWithStaticNodes); + } + + @Test + public void testClose() { + client.ready(node, time.milliseconds()); + awaitReady(client, node); + client.poll(1, time.milliseconds()); + assertTrue("The client should be ready", client.isReady(node, time.milliseconds())); + ProduceRequest produceRequest = new ProduceRequest((short) 1, 1000, Collections.emptyMap()); RequestHeader reqHeader = client.nextRequestHeader(ApiKeys.PRODUCE); - RequestSend send = new RequestSend(node.id(), reqHeader, produceRequest.toStruct()); + RequestSend send = new RequestSend(node.idString(), reqHeader, produceRequest.toStruct()); ClientRequest request = new ClientRequest(time.milliseconds(), true, send, null); - awaitReady(client, node); - client.poll(Arrays.asList(request), 1, time.milliseconds()); - assertEquals(1, client.inFlightRequestCount()); + client.send(request, time.milliseconds()); + assertEquals("There should be 1 in-flight request after send", 1, client.inFlightRequestCount(node.idString())); + + client.close(node.idString()); + assertEquals("There should be no in-flight request after close", 0, client.inFlightRequestCount(node.idString())); + assertFalse("Connection should not be ready after close", client.isReady(node, 0)); + } + + private void checkSimpleRequestResponse(NetworkClient networkClient) { + ProduceRequest produceRequest = new ProduceRequest((short) 1, 1000, Collections.emptyMap()); + RequestHeader reqHeader = networkClient.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(networkClient, node); + networkClient.send(request, time.milliseconds()); + networkClient.poll(1, time.milliseconds()); + assertEquals(1, networkClient.inFlightRequestCount()); ResponseHeader respHeader = new ResponseHeader(reqHeader.correlationId()); Struct resp = new Struct(ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id)); resp.set("responses", new Object[0]); @@ -85,17 +119,43 @@ public void testSimpleRequestResponse() { respHeader.writeTo(buffer); resp.writeTo(buffer); buffer.flip(); - selector.completeReceive(new NetworkReceive(node.id(), buffer)); - List responses = client.poll(new ArrayList(), 1, time.milliseconds()); + selector.completeReceive(new NetworkReceive(node.idString(), buffer)); + List responses = networkClient.poll(1, time.milliseconds()); assertEquals(1, responses.size()); - ClientResponse response = responses.get(0); - assertTrue("Should have a response body.", response.hasResponse()); - assertEquals("Should be correlated to the original request", request, response.request()); + 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(new ArrayList(), 1, time.milliseconds()); + client.poll(1, time.milliseconds()); + } + + @Test + public void testRequestTimeout() { + 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); + long now = time.milliseconds(); + client.send(request, now); + // sleeping to make sure that the time since last send is greater than requestTimeOut + time.sleep(3000); + client.poll(3000, time.milliseconds()); + String disconnectedNode = selector.disconnected().get(0); + assertEquals(node.idString(), disconnectedNode); } + 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/ConsumerExampleTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerExampleTest.java deleted file mode 100644 index 29ad25e90606f..0000000000000 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerExampleTest.java +++ /dev/null @@ -1,297 +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.consumer; - -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Properties; - -import org.apache.kafka.common.TopicPartition; -import org.junit.Test; - -/** - * TODO: Clean this after the consumer implementation is complete. Until then, it is useful to write some sample test code using the new APIs - * - */ -public class ConsumerExampleTest { - /** - * This example demonstrates how to use the consumer to leverage Kafka's group management functionality for automatic consumer load - * balancing and failure detection. This example assumes that the offsets are stored in Kafka and are automatically committed periodically, - * as controlled by the auto.commit.interval.ms config - */ -// @Test -// public void testConsumerGroupManagementWithAutoOffsetCommits() { -// Properties props = new Properties(); -// props.put("metadata.broker.list", "localhost:9092"); -// props.put("group.id", "test"); -// props.put("session.timeout.ms", "1000"); -// props.put("auto.commit.enable", "true"); -// props.put("auto.commit.interval.ms", "10000"); -// KafkaConsumer consumer = new KafkaConsumer(props); -// // subscribe to some topics -// consumer.subscribe("foo", "bar"); -// boolean isRunning = true; -// while(isRunning) { -// Map records = consumer.poll(100); -// process(records); -// } -// consumer.close(); -// } - - /** - * This example demonstrates how to use the consumer to leverage Kafka's group management functionality for automatic consumer load - * balancing and failure detection. This example assumes that the offsets are stored in Kafka and are manually committed using the - * commit() API. This example also demonstrates rewinding the consumer's offsets if processing of consumed messages fails. - */ -// @Test -// public void testConsumerGroupManagementWithManualOffsetCommit() { -// Properties props = new Properties(); -// props.put("metadata.broker.list", "localhost:9092"); -// props.put("group.id", "test"); -// props.put("session.timeout.ms", "1000"); -// props.put("auto.commit.enable", "false"); -// KafkaConsumer consumer = new KafkaConsumer(props); -// // subscribe to some topics -// consumer.subscribe("foo", "bar"); -// int commitInterval = 100; -// int numRecords = 0; -// boolean isRunning = true; -// Map consumedOffsets = new HashMap(); -// while(isRunning) { -// Map records = consumer.poll(100); -// try { -// Map lastConsumedOffsets = process(records); -// consumedOffsets.putAll(lastConsumedOffsets); -// numRecords += records.size(); -// // commit offsets for all partitions of topics foo, bar synchronously, owned by this consumer instance -// if(numRecords % commitInterval == 0) -// consumer.commit(true); -// } catch(Exception e) { -// // rewind consumer's offsets for failed partitions -// List failedPartitions = getFailedPartitions(); -// Map offsetsToRewindTo = new HashMap(); -// for(TopicPartition failedPartition : failedPartitions) { -// // rewind to the last consumed offset for the failed partition. Since process() failed for this partition, the consumed offset -// // should still be pointing to the last successfully processed offset and hence is the right offset to rewind consumption to. -// offsetsToRewindTo.put(failedPartition, consumedOffsets.get(failedPartition)); -// } -// // seek to new offsets only for partitions that failed the last process() -// consumer.seek(offsetsToRewindTo); -// } -// } -// consumer.close(); -// } - - private List getFailedPartitions() { return null; } - - /** - * This example demonstrates the consumer can be used to leverage Kafka's group management functionality along with custom offset storage. - * In this example, the assumption made is that the user chooses to store the consumer offsets outside Kafka. This requires the user to - * plugin logic for retrieving the offsets from a custom store and provide the offsets to the consumer in the ConsumerRebalanceCallback - * callback. The onPartitionsAssigned callback is invoked after the consumer is assigned a new set of partitions on rebalance and - * before the consumption restarts post rebalance. This is the right place to supply offsets from a custom store to the consumer. - */ -// @Test -// public void testConsumerRebalanceWithCustomOffsetStore() { -// Properties props = new Properties(); -// props.put("metadata.broker.list", "localhost:9092"); -// props.put("group.id", "test"); -// props.put("session.timeout.ms", "1000"); -// props.put("auto.commit.enable", "true"); -// props.put("auto.commit.interval.ms", "10000"); -// KafkaConsumer consumer = new KafkaConsumer(props, -// new ConsumerRebalanceCallback() { -// public void onPartitionsAssigned(Consumer consumer, Collection partitions) { -// Map lastCommittedOffsets = getLastCommittedOffsetsFromCustomStore(partitions); -// consumer.seek(lastCommittedOffsets); -// } -// public void onPartitionsRevoked(Consumer consumer, Collection partitions) { -// Map offsets = getLastConsumedOffsets(partitions); // implemented by the user -// commitOffsetsToCustomStore(offsets); // implemented by the user -// } -// private Map getLastCommittedOffsetsFromCustomStore(Collection partitions) { -// return null; -// } -// private Map getLastConsumedOffsets(Collection partitions) { return null; } -// private void commitOffsetsToCustomStore(Map offsets) {} -// }); -// // subscribe to topics -// consumer.subscribe("foo", "bar"); -// int commitInterval = 100; -// int numRecords = 0; -// boolean isRunning = true; -// while(isRunning) { -// Map records = consumer.poll(100); -// Map consumedOffsets = process(records); -// numRecords += records.size(); -// // commit offsets for all partitions of topics foo, bar synchronously, owned by this consumer instance -// if(numRecords % commitInterval == 0) -// commitOffsetsToCustomStore(consumedOffsets); -// } -// consumer.close(); -// } - - /** - * This example demonstrates how the consumer can be used to leverage Kafka's group management functionality along with Kafka based offset storage. - * In this example, the assumption made is that the user chooses to use Kafka based offset management. - */ -// @Test -// public void testConsumerRewindWithGroupManagementAndKafkaOffsetStorage() { -// Properties props = new Properties(); -// props.put("metadata.broker.list", "localhost:9092"); -// props.put("group.id", "test"); -// props.put("session.timeout.ms", "1000"); -// props.put("auto.commit.enable", "false"); -// KafkaConsumer consumer = new KafkaConsumer(props, -// new ConsumerRebalanceCallback() { -// boolean rewindOffsets = true; -// public void onPartitionsAssigned(Consumer consumer, Collection partitions) { -// if(rewindOffsets) { -// Map latestCommittedOffsets = consumer.committed(null); -// Map newOffsets = rewindOffsets(latestCommittedOffsets, 100); -// consumer.seek(newOffsets); -// } -// } -// public void onPartitionsRevoked(Consumer consumer, Collection partitions) { -// consumer.commit(true); -// } -// // this API rewinds every partition back by numberOfMessagesToRewindBackTo messages -// private Map rewindOffsets(Map currentOffsets, -// long numberOfMessagesToRewindBackTo) { -// Map newOffsets = new HashMap(); -// for(Map.Entry offset : currentOffsets.entrySet()) { -// newOffsets.put(offset.getKey(), offset.getValue() - numberOfMessagesToRewindBackTo); -// } -// return newOffsets; -// } -// }); -// // subscribe to topics -// consumer.subscribe("foo", "bar"); -// int commitInterval = 100; -// int numRecords = 0; -// boolean isRunning = true; -// while(isRunning) { -// Map records = consumer.poll(100); -// Map consumedOffsets = process(records); -// numRecords += records.size(); -// // commit offsets for all partitions of topics foo, bar synchronously, owned by this consumer instance -// if(numRecords % commitInterval == 0) -// commitOffsetsToCustomStore(consumedOffsets); -// } -// consumer.close(); -// } - - /** - * This example demonstrates how the consumer can be used to subscribe to specific partitions of certain topics and consume upto the latest - * available message for each of those partitions before shutting down. When used to subscribe to specific partitions, the user foregoes - * the group management functionality and instead relies on manually configuring the consumer instances to subscribe to a set of partitions. - * This example assumes that the user chooses to use Kafka based offset storage. The user still has to specify a group.id to use Kafka - * based offset management. However, session.timeout.ms is not required since the Kafka consumer only does failure detection with group - * management. - */ -// @Test -// public void testConsumerWithKafkaBasedOffsetManagement() { -// Properties props = new Properties(); -// props.put("metadata.broker.list", "localhost:9092"); -// props.put("group.id", "test"); -// props.put("auto.commit.enable", "true"); -// props.put("auto.commit.interval.ms", "10000"); -// KafkaConsumer consumer = new KafkaConsumer(props); -// // subscribe to some partitions of topic foo -// TopicPartition partition0 = new TopicPartition("foo", 0); -// TopicPartition partition1 = new TopicPartition("foo", 1); -// TopicPartition[] partitions = new TopicPartition[2]; -// partitions[0] = partition0; -// partitions[1] = partition1; -// consumer.subscribe(partitions); -// // find the last committed offsets for partitions 0,1 of topic foo -// Map lastCommittedOffsets = consumer.committed(null); -// // seek to the last committed offsets to avoid duplicates -// consumer.seek(lastCommittedOffsets); -// // find the offsets of the latest available messages to know where to stop consumption -// Map latestAvailableOffsets = consumer.offsetsBeforeTime(-2, null); -// boolean isRunning = true; -// while(isRunning) { -// Map records = consumer.poll(100); -// Map consumedOffsets = process(records); -// for(TopicPartition partition : partitions) { -// if(consumedOffsets.get(partition) >= latestAvailableOffsets.get(partition)) -// isRunning = false; -// else -// isRunning = true; -// } -// } -// consumer.close(); -// } - - /** - * This example demonstrates how the consumer can be used to subscribe to specific partitions of certain topics and consume upto the latest - * available message for each of those partitions before shutting down. When used to subscribe to specific partitions, the user foregoes - * the group management functionality and instead relies on manually configuring the consumer instances to subscribe to a set of partitions. - * This example assumes that the user chooses to use custom offset storage. - */ - @Test - public void testConsumerWithCustomOffsetManagement() { -// Properties props = new Properties(); -// props.put("metadata.broker.list", "localhost:9092"); -// KafkaConsumer consumer = new KafkaConsumer(props); -// // subscribe to some partitions of topic foo -// TopicPartition partition0 = new TopicPartition("foo", 0); -// TopicPartition partition1 = new TopicPartition("foo", 1); -// TopicPartition[] partitions = new TopicPartition[2]; -// partitions[0] = partition0; -// partitions[1] = partition1; -// consumer.subscribe(partitions); -// Map lastCommittedOffsets = getLastCommittedOffsetsFromCustomStore(); -// // seek to the last committed offsets to avoid duplicates -// consumer.seek(lastCommittedOffsets); -// // find the offsets of the latest available messages to know where to stop consumption -// Map latestAvailableOffsets = consumer.offsetsBeforeTime(-2, null); -// boolean isRunning = true; -// while(isRunning) { -// Map records = consumer.poll(100); -// Map consumedOffsets = process(records); -// // commit offsets for partitions 0,1 for topic foo to custom store -// commitOffsetsToCustomStore(consumedOffsets); -// for(TopicPartition partition : partitions) { -// if(consumedOffsets.get(partition) >= latestAvailableOffsets.get(partition)) -// isRunning = false; -// else -// isRunning = true; -// } -// } -// consumer.close(); - } - - private Map getLastCommittedOffsetsFromCustomStore() { return null; } - private void commitOffsetsToCustomStore(Map consumedOffsets) {} - private Map process(Map records) { - Map processedOffsets = new HashMap(); - for(Entry recordMetadata : records.entrySet()) { - List recordsPerTopic = recordMetadata.getValue().records(); - for(int i = 0;i < recordsPerTopic.size();i++) { - ConsumerRecord record = recordsPerTopic.get(i); - // process record - try { - processedOffsets.put(record.topicAndPartition(), record.offset()); - } catch (Exception e) { - e.printStackTrace(); - } - } - } - return processedOffsets; - } -} 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..7625218422ae7 --- /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, 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..fa06be9e2ba2a --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.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.clients.consumer; + +import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener; +import org.apache.kafka.common.TopicPartition; +import org.junit.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + +public class MockConsumerTest { + + private MockConsumer consumer = new MockConsumer(OffsetResetStrategy.EARLIEST); + + @Test + public void testSimpleMock() { + consumer.subscribe(Arrays.asList("test"), new NoOpConsumerRebalanceListener()); + assertEquals(0, consumer.poll(1000).count()); + consumer.rebalance(Arrays.asList(new TopicPartition("test", 0), new TopicPartition("test", 1))); + // Mock consumers need to seek manually since they cannot automatically reset offsets + HashMap beginningOffsets = new HashMap<>(); + beginningOffsets.put(new TopicPartition("test", 0), 0L); + beginningOffsets.put(new TopicPartition("test", 1), 0L); + consumer.updateBeginningOffsets(beginningOffsets); + consumer.seek(new TopicPartition("test", 0), 0); + 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(2L, consumer.position(new TopicPartition("test", 0))); + consumer.commitSync(); + assertEquals(2L, consumer.committed(new TopicPartition("test", 0)).offset()); + } + +} 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..66b2e32f402fc --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java @@ -0,0 +1,635 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.ClientRequest; +import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.MockClient; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.OffsetCommitCallback; +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.errors.OffsetMetadataTooLarge; +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.OffsetCommitRequest; +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.Arrays; +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.After; +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 int heartbeatIntervalMs = 2; + private long retryBackoffMs = 100; + private long requestTimeoutMs = 5000; + private boolean autoCommitEnabled = false; + private long autoCommitIntervalMs = 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 MockRebalanceListener subscriptionListener; + private MockCommitCallback defaultOffsetCommitCallback; + 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.subscriptionListener = new MockRebalanceListener(); + this.defaultOffsetCommitCallback = new MockCommitCallback(); + + client.setNode(node); + + this.coordinator = new Coordinator(consumerClient, + groupId, + sessionTimeoutMs, + heartbeatIntervalMs, + rebalanceStrategy, + subscriptions, + metrics, + "consumer" + groupId, + metricTags, + time, + requestTimeoutMs, + retryBackoffMs, + defaultOffsetCommitCallback, + autoCommitEnabled, + autoCommitIntervalMs); + } + + @After + public void teardown() { + this.metrics.close(); + } + + @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(Arrays.asList(topicName), subscriptionListener); + 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(Arrays.asList(topicName), subscriptionListener); + 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(Arrays.asList(topicName), subscriptionListener); + 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, subscriptionListener.revokedCount); + assertEquals(Collections.emptySet(), subscriptionListener.revoked); + assertEquals(1, subscriptionListener.assignedCount); + assertEquals(Collections.singleton(tp), subscriptionListener.assigned); + } + + @Test + public void testReJoinGroup() { + subscriptions.subscribe(Arrays.asList(topicName), subscriptionListener); + 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, subscriptionListener.revokedCount); + assertEquals(Collections.emptySet(), subscriptionListener.revoked); + assertEquals(1, subscriptionListener.assignedCount); + assertEquals(Collections.singleton(tp), subscriptionListener.assigned); + } + + @Test(expected = ApiException.class) + public void testUnknownPartitionAssignmentStrategy() { + subscriptions.subscribe(Arrays.asList(topicName), subscriptionListener); + 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(Arrays.asList(topicName), subscriptionListener); + 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 testCommitOffsetOnly() { + subscriptions.assign(Arrays.asList(tp)); + + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + + AtomicBoolean success = new AtomicBoolean(false); + coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), callback(success)); + consumerClient.poll(0); + assertTrue(success.get()); + + assertEquals(100L, subscriptions.committed(tp).offset()); + } + + @Test + public void testCommitOffsetMetadata() { + subscriptions.assign(Arrays.asList(tp)); + + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + + AtomicBoolean success = new AtomicBoolean(false); + coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L, "hello")), callback(success)); + consumerClient.poll(0); + assertTrue(success.get()); + + assertEquals(100L, subscriptions.committed(tp).offset()); + assertEquals("hello", subscriptions.committed(tp).metadata()); + } + + @Test + public void testCommitOffsetAsyncWithDefaultCallback() { + int invokedBeforeTest = defaultOffsetCommitCallback.invoked; + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), null); + consumerClient.poll(0); + assertEquals(invokedBeforeTest + 1, defaultOffsetCommitCallback.invoked); + assertNull(defaultOffsetCommitCallback.exception); + } + + @Test + public void testResetGeneration() { + // enable auto-assignment + subscriptions.subscribe(Arrays.asList(topicName), subscriptionListener); + + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code())); + coordinator.ensurePartitionAssignment(); + + // now switch to manual assignment + subscriptions.unsubscribe(); + coordinator.resetGeneration(); + subscriptions.assign(Arrays.asList(tp)); + + // the client should not reuse generation/consumerId from auto-subscribed generation + client.prepareResponse(new MockClient.RequestMatcher() { + @Override + public boolean matches(ClientRequest request) { + OffsetCommitRequest commitRequest = new OffsetCommitRequest(request.request().body()); + return commitRequest.consumerId().equals(OffsetCommitRequest.DEFAULT_CONSUMER_ID) && + commitRequest.generationId() == OffsetCommitRequest.DEFAULT_GENERATION_ID; + } + }, offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + + AtomicBoolean success = new AtomicBoolean(false); + coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), callback(success)); + consumerClient.poll(0); + assertTrue(success.get()); + } + + @Test + public void testCommitOffsetAsyncFailedWithDefaultCallback() { + int invokedBeforeTest = defaultOffsetCommitCallback.invoked; + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()))); + coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), null); + consumerClient.poll(0); + assertEquals(invokedBeforeTest + 1, defaultOffsetCommitCallback.invoked); + assertEquals(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.exception(), defaultOffsetCommitCallback.exception); + } + + @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.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), 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.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), 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.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), 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) + 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.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L))); + } + + @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) + 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.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L))); + } + + @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) + 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.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L))); + } + + @Test(expected = OffsetMetadataTooLarge.class) + public void testCommitOffsetMetadataTooLarge() { + // since offset metadata is provided by the user, we have to propagate the exception so they can handle it + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.OFFSET_METADATA_TOO_LARGE.code()))); + coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L, "metadata"))); + } + + @Test(expected = ApiException.class) + public void testCommitOffsetSyncCallbackWithNonRetriableException() { + 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.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L))); + } + + @Test + public void testRefreshOffset() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + subscriptions.assign(Arrays.asList(tp)); + subscriptions.needRefreshCommits(); + client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); + coordinator.refreshCommittedOffsetsIfNeeded(); + assertFalse(subscriptions.refreshCommitsNeeded()); + assertEquals(100L, subscriptions.committed(tp).offset()); + } + + @Test + public void testRefreshOffsetLoadInProgress() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + subscriptions.assign(Arrays.asList(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, subscriptions.committed(tp).offset()); + } + + @Test + public void testRefreshOffsetNotCoordinatorForConsumer() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + subscriptions.assign(Arrays.asList(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, subscriptions.committed(tp).offset()); + } + + @Test + public void testRefreshOffsetWithNoFetchableOffsets() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + subscriptions.assign(Arrays.asList(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 OffsetCommitCallback callback(final AtomicBoolean success) { + return new OffsetCommitCallback() { + @Override + public void onComplete(Map offsets, Exception exception) { + if (exception == null) + success.set(true); + } + }; + } + + private static class MockCommitCallback implements OffsetCommitCallback { + public int invoked = 0; + public Exception exception = null; + + @Override + public void onComplete(Map offsets, Exception exception) { + invoked++; + this.exception = exception; + } + } + + private static class MockRebalanceListener implements ConsumerRebalanceListener { + 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..f5f9ef1c86e4f --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -0,0 +1,408 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.Metadata; +import org.apache.kafka.clients.MockClient; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +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.OffsetOutOfRangeException; +import org.apache.kafka.common.metrics.KafkaMetric; +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.requests.ListOffsetRequest; +import org.apache.kafka.common.requests.ListOffsetResponse; +import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +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 static org.junit.Assert.fail; + +public class FetcherTest { + private ConsumerRebalanceListener listener = new NoOpConsumerRebalanceListener(); + private String topicName = "test"; + private String groupId = "test-group"; + private final String metricGroup = "consumer" + groupId + "-fetch-manager-metrics"; + 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 SubscriptionState subscriptionsNoAutoReset = new SubscriptionState(OffsetResetStrategy.NONE); + private Metrics metrics = new Metrics(time); + private Map metricTags = new LinkedHashMap(); + private static final double EPSILON = 0.0001; + private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100); + + private MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE); + private Fetcher fetcher = createFetcher(subscriptions, metrics); + private Metrics fetcherMetrics = new Metrics(time); + private Fetcher fetcherNoAutoReset = createFetcher(subscriptionsNoAutoReset, fetcherMetrics); + + @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(); + } + + @After + public void teardown() { + this.metrics.close(); + this.fetcherMetrics.close(); + } + + @Test + public void testFetchNormal() { + List> records; + subscriptions.assign(Arrays.asList(tp)); + subscriptions.seek(tp, 0); + + // normal fetch + fetcher.initFetches(cluster); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0)); + 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 testFetchDuringRebalance() { + subscriptions.subscribe(Arrays.asList(topicName), listener); + subscriptions.changePartitionAssignment(Arrays.asList(tp)); + subscriptions.seek(tp, 0); + + fetcher.initFetches(cluster); + + // Now the rebalance happens and fetch positions are cleared + subscriptions.changePartitionAssignment(Arrays.asList(tp)); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0)); + consumerClient.poll(0); + + // The active fetch should be ignored since its position is no longer valid + assertTrue(fetcher.fetchedRecords().isEmpty()); + } + + @Test + public void testInFlightFetchOnPausedPartition() { + subscriptions.assign(Arrays.asList(tp)); + subscriptions.seek(tp, 0); + + fetcher.initFetches(cluster); + subscriptions.pause(tp); + + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0)); + consumerClient.poll(0); + assertNull(fetcher.fetchedRecords().get(tp)); + } + + @Test + public void testFetchOnPausedPartition() { + subscriptions.assign(Arrays.asList(tp)); + subscriptions.seek(tp, 0); + + subscriptions.pause(tp); + fetcher.initFetches(cluster); + assertTrue(client.requests().isEmpty()); + } + + @Test + public void testFetchNotLeaderForPartition() { + subscriptions.assign(Arrays.asList(tp)); + subscriptions.seek(tp, 0); + + fetcher.initFetches(cluster); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NOT_LEADER_FOR_PARTITION.code(), 100L, 0)); + consumerClient.poll(0); + assertEquals(0, fetcher.fetchedRecords().size()); + assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds())); + } + + @Test + public void testFetchUnknownTopicOrPartition() { + subscriptions.assign(Arrays.asList(tp)); + subscriptions.seek(tp, 0); + + fetcher.initFetches(cluster); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), 100L, 0)); + consumerClient.poll(0); + assertEquals(0, fetcher.fetchedRecords().size()); + assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds())); + } + + @Test + public void testFetchOffsetOutOfRange() { + subscriptions.assign(Arrays.asList(tp)); + subscriptions.seek(tp, 0); + + fetcher.initFetches(cluster); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0)); + 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 testFetchedRecordsAfterSeek() { + subscriptionsNoAutoReset.assign(Arrays.asList(tp)); + subscriptionsNoAutoReset.seek(tp, 0); + + fetcherNoAutoReset.initFetches(cluster); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0)); + consumerClient.poll(0); + assertFalse(subscriptionsNoAutoReset.isOffsetResetNeeded(tp)); + subscriptionsNoAutoReset.seek(tp, 2); + assertEquals(0, fetcherNoAutoReset.fetchedRecords().size()); + } + + @Test + public void testFetchOffsetOutOfRangeException() { + subscriptionsNoAutoReset.assign(Arrays.asList(tp)); + subscriptionsNoAutoReset.seek(tp, 0); + + fetcherNoAutoReset.initFetches(cluster); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0)); + consumerClient.poll(0); + assertFalse(subscriptionsNoAutoReset.isOffsetResetNeeded(tp)); + try { + fetcherNoAutoReset.fetchedRecords(); + fail("Should have thrown OffsetOutOfRangeException"); + } catch (OffsetOutOfRangeException e) { + assertTrue(e.offsetOutOfRangePartitions().containsKey(tp)); + assertEquals(e.offsetOutOfRangePartitions().size(), 1); + } + assertEquals(0, fetcherNoAutoReset.fetchedRecords().size()); + } + + @Test + public void testFetchDisconnected() { + subscriptions.assign(Arrays.asList(tp)); + subscriptions.seek(tp, 0); + + fetcher.initFetches(cluster); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0), true); + consumerClient.poll(0); + assertEquals(0, fetcher.fetchedRecords().size()); + + // disconnects should have no affect on subscription state + assertFalse(subscriptions.isOffsetResetNeeded(tp)); + assertTrue(subscriptions.isFetchable(tp)); + assertEquals(0, (long) subscriptions.fetched(tp)); + assertEquals(0, (long) subscriptions.consumed(tp)); + } + + @Test + public void testUpdateFetchPositionToCommitted() { + // unless a specific reset is expected, the default behavior is to reset to the committed + // position if one is present + subscriptions.assign(Arrays.asList(tp)); + subscriptions.committed(tp, new OffsetAndMetadata(5)); + + fetcher.updateFetchPositions(Collections.singleton(tp)); + assertTrue(subscriptions.isFetchable(tp)); + assertEquals(5, (long) subscriptions.fetched(tp)); + assertEquals(5, (long) subscriptions.consumed(tp)); + } + + @Test + public void testUpdateFetchPositionResetToDefaultOffset() { + subscriptions.assign(Arrays.asList(tp)); + // with no commit position, we should reset using the default strategy defined above (EARLIEST) + + client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.EARLIEST_TIMESTAMP), + listOffsetResponse(Errors.NONE, Arrays.asList(5L))); + fetcher.updateFetchPositions(Collections.singleton(tp)); + assertFalse(subscriptions.isOffsetResetNeeded(tp)); + assertTrue(subscriptions.isFetchable(tp)); + assertEquals(5, (long) subscriptions.fetched(tp)); + assertEquals(5, (long) subscriptions.consumed(tp)); + } + + @Test + public void testUpdateFetchPositionResetToLatestOffset() { + subscriptions.assign(Arrays.asList(tp)); + subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST); + + client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.LATEST_TIMESTAMP), + listOffsetResponse(Errors.NONE, Arrays.asList(5L))); + fetcher.updateFetchPositions(Collections.singleton(tp)); + assertFalse(subscriptions.isOffsetResetNeeded(tp)); + assertTrue(subscriptions.isFetchable(tp)); + assertEquals(5, (long) subscriptions.fetched(tp)); + assertEquals(5, (long) subscriptions.consumed(tp)); + } + + @Test + public void testUpdateFetchPositionResetToEarliestOffset() { + subscriptions.assign(Arrays.asList(tp)); + subscriptions.needOffsetReset(tp, OffsetResetStrategy.EARLIEST); + + client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.EARLIEST_TIMESTAMP), + listOffsetResponse(Errors.NONE, Arrays.asList(5L))); + fetcher.updateFetchPositions(Collections.singleton(tp)); + assertFalse(subscriptions.isOffsetResetNeeded(tp)); + assertTrue(subscriptions.isFetchable(tp)); + assertEquals(5, (long) subscriptions.fetched(tp)); + assertEquals(5, (long) subscriptions.consumed(tp)); + } + + @Test + public void testUpdateFetchPositionDisconnect() { + subscriptions.assign(Arrays.asList(tp)); + subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST); + + // First request gets a disconnect + client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.LATEST_TIMESTAMP), + listOffsetResponse(Errors.NONE, Arrays.asList(5L)), true); + + // Next one succeeds + client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.LATEST_TIMESTAMP), + listOffsetResponse(Errors.NONE, Arrays.asList(5L))); + fetcher.updateFetchPositions(Collections.singleton(tp)); + assertFalse(subscriptions.isOffsetResetNeeded(tp)); + assertTrue(subscriptions.isFetchable(tp)); + assertEquals(5, (long) subscriptions.fetched(tp)); + assertEquals(5, (long) subscriptions.consumed(tp)); + } + + @Test + public void testGetAllTopics() throws InterruptedException { + // sending response before request, as getAllTopics is a blocking call + client.prepareResponse( + new MetadataResponse(cluster, Collections.emptyMap()).toStruct()); + + Map> allTopics = fetcher.getAllTopics(5000L); + + assertEquals(cluster.topics().size(), allTopics.size()); + } + + /* + * Send multiple requests. Verify that the client side quota metrics have the right values + */ + @Test + public void testQuotaMetrics() throws Exception { + List> records; + subscriptions.assign(Arrays.asList(tp)); + subscriptions.seek(tp, 0); + + // normal fetch + for (int i = 1; i < 4; i++) { + fetcher.initFetches(cluster); + + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 100 * i)); + consumerClient.poll(0); + records = fetcher.fetchedRecords().get(tp); + assertEquals(3, records.size()); + } + + Map allMetrics = metrics.metrics(); + KafkaMetric avgMetric = allMetrics.get(new MetricName("fetch-throttle-time-avg", metricGroup, "", metricTags)); + KafkaMetric maxMetric = allMetrics.get(new MetricName("fetch-throttle-time-max", metricGroup, "", metricTags)); + assertEquals(200, avgMetric.value(), EPSILON); + assertEquals(300, maxMetric.value(), EPSILON); + } + + private MockClient.RequestMatcher listOffsetRequestMatcher(final long timestamp) { + // matches any list offset request with the provided timestamp + return new MockClient.RequestMatcher() { + @Override + public boolean matches(ClientRequest request) { + ListOffsetRequest req = new ListOffsetRequest(request.request().body()); + ListOffsetRequest.PartitionData partitionData = req.offsetData().get(tp); + return partitionData != null && partitionData.timestamp == timestamp; + } + }; + } + + private Struct listOffsetResponse(Errors error, List offsets) { + ListOffsetResponse.PartitionData partitionData = new ListOffsetResponse.PartitionData(error.code(), offsets); + Map allPartitionData = new HashMap<>(); + allPartitionData.put(tp, partitionData); + ListOffsetResponse response = new ListOffsetResponse(allPartitionData); + return response.toStruct(); + } + + private Struct fetchResponse(ByteBuffer buffer, short error, long hw, int throttleTime) { + FetchResponse response = new FetchResponse(Collections.singletonMap(tp, new FetchResponse.PartitionData(error, hw, buffer)), throttleTime); + return response.toStruct(); + } + + private Fetcher createFetcher(SubscriptionState subscriptions, Metrics metrics) { + return new Fetcher(consumerClient, + minBytes, + maxWaitMs, + fetchSize, + true, // check crc + new ByteArrayDeserializer(), + new ByteArrayDeserializer(), + metadata, + subscriptions, + metrics, + "consumer" + groupId, + metricTags, + time, + retryBackoffMs); + } +} 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..75e68cc56ba11 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.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.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 long interval = 100L; + private MockTime time = new MockTime(); + private Heartbeat heartbeat = new Heartbeat(timeout, interval, -1L); + + @Test + public void testShouldHeartbeat() { + heartbeat.sentHeartbeat(time.milliseconds()); + time.sleep((long) ((float) interval * 1.1)); + assertTrue(heartbeat.shouldHeartbeat(time.milliseconds())); + } + + @Test + public void testShouldNotHeartbeat() { + heartbeat.sentHeartbeat(time.milliseconds()); + time.sleep(interval / 2); + 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..a0568add53cd6 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java @@ -0,0 +1,226 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.assertTrue; +import static java.util.Arrays.asList; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.regex.Pattern; + +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +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); + private final MockRebalanceListener rebalanceListener = new MockRebalanceListener(); + + @Test + public void partitionAssignment() { + state.assign(Arrays.asList(tp0)); + assertEquals(Collections.singleton(tp0), state.assignedPartitions()); + state.committed(tp0, new OffsetAndMetadata(1)); + state.seek(tp0, 1); + assertTrue(state.isFetchable(tp0)); + assertAllPositions(tp0, 1L); + state.assign(Arrays.asList()); + assertTrue(state.assignedPartitions().isEmpty()); + assertFalse(state.isAssigned(tp0)); + assertFalse(state.isFetchable(tp0)); + } + + @Test + public void partitionReset() { + state.assign(Arrays.asList(tp0)); + state.seek(tp0, 5); + assertEquals(5L, (long) state.fetched(tp0)); + assertEquals(5L, (long) state.consumed(tp0)); + state.needOffsetReset(tp0); + assertFalse(state.isFetchable(tp0)); + assertTrue(state.isOffsetResetNeeded(tp0)); + assertEquals(null, state.fetched(tp0)); + assertEquals(null, state.consumed(tp0)); + + // seek should clear the reset and make the partition fetchable + state.seek(tp0, 0); + assertTrue(state.isFetchable(tp0)); + assertFalse(state.isOffsetResetNeeded(tp0)); + } + + @Test + public void topicSubscription() { + state.subscribe(Arrays.asList("test"), rebalanceListener); + assertEquals(1, state.subscription().size()); + assertTrue(state.assignedPartitions().isEmpty()); + assertTrue(state.partitionsAutoAssigned()); + state.changePartitionAssignment(asList(tp0)); + state.seek(tp0, 1); + state.committed(tp0, new OffsetAndMetadata(1)); + assertAllPositions(tp0, 1L); + state.changePartitionAssignment(asList(tp1)); + assertTrue(state.isAssigned(tp1)); + assertFalse(state.isAssigned(tp0)); + assertFalse(state.isFetchable(tp1)); + assertEquals(Collections.singleton(tp1), state.assignedPartitions()); + } + + @Test + public void partitionPause() { + state.assign(Arrays.asList(tp0)); + state.seek(tp0, 100); + assertTrue(state.isFetchable(tp0)); + state.pause(tp0); + assertFalse(state.isFetchable(tp0)); + state.resume(tp0); + assertTrue(state.isFetchable(tp0)); + } + + @Test + public void commitOffsetMetadata() { + state.assign(Arrays.asList(tp0)); + state.committed(tp0, new OffsetAndMetadata(5, "hi")); + + assertEquals(5, state.committed(tp0).offset()); + assertEquals("hi", state.committed(tp0).metadata()); + } + + @Test + public void topicUnsubscription() { + final String topic = "test"; + state.subscribe(Arrays.asList(topic), rebalanceListener); + assertEquals(1, state.subscription().size()); + assertTrue(state.assignedPartitions().isEmpty()); + assertTrue(state.partitionsAutoAssigned()); + state.changePartitionAssignment(asList(tp0)); + state.committed(tp0, new OffsetAndMetadata(1)); + state.seek(tp0, 1); + assertAllPositions(tp0, 1L); + state.changePartitionAssignment(asList(tp1)); + assertFalse(state.isAssigned(tp0)); + assertEquals(Collections.singleton(tp1), state.assignedPartitions()); + + state.subscribe(Arrays.asList(), rebalanceListener); + assertEquals(0, state.subscription().size()); + assertTrue(state.assignedPartitions().isEmpty()); + } + + @Test(expected = IllegalStateException.class) + public void invalidConsumedPositionUpdate() { + state.subscribe(Arrays.asList("test"), rebalanceListener); + state.changePartitionAssignment(asList(tp0)); + state.consumed(tp0, 0); + } + + @Test(expected = IllegalStateException.class) + public void invalidFetchPositionUpdate() { + state.subscribe(Arrays.asList("test"), rebalanceListener); + state.changePartitionAssignment(asList(tp0)); + state.fetched(tp0, 0); + } + + @Test(expected = IllegalStateException.class) + public void cantChangeFetchPositionForNonAssignedPartition() { + state.fetched(tp0, 1); + } + + @Test(expected = IllegalStateException.class) + public void cantChangeConsumedPositionForNonAssignedPartition() { + state.consumed(tp0, 1); + } + + public void assertAllPositions(TopicPartition tp, Long offset) { + assertEquals(offset.longValue(), state.committed(tp).offset()); + assertEquals(offset, state.fetched(tp)); + assertEquals(offset, state.consumed(tp)); + } + + @Test(expected = IllegalStateException.class) + public void cantSubscribeTopicAndPattern() { + state.subscribe(Arrays.asList("test"), rebalanceListener); + state.subscribe(Pattern.compile(".*"), rebalanceListener); + } + + @Test(expected = IllegalStateException.class) + public void cantSubscribePartitionAndPattern() { + state.assign(Arrays.asList(new TopicPartition("test", 0))); + state.subscribe(Pattern.compile(".*"), rebalanceListener); + } + + @Test(expected = IllegalStateException.class) + public void cantSubscribePatternAndTopic() { + state.subscribe(Pattern.compile(".*"), rebalanceListener); + state.subscribe(Arrays.asList("test"), rebalanceListener); + } + + @Test(expected = IllegalStateException.class) + public void cantSubscribePatternAndPartition() { + state.subscribe(Pattern.compile(".*"), rebalanceListener); + state.assign(Arrays.asList(new TopicPartition("test", 0))); + } + + @Test + public void patternSubscription() { + state.subscribe(Pattern.compile(".*"), rebalanceListener); + state.changeSubscription(Arrays.asList("test", "test1")); + + assertEquals( + "Expected subscribed topics count is incorrect", 2, state.subscription().size()); + } + + @Test + public void patternUnsubscription() { + state.subscribe(Pattern.compile(".*"), rebalanceListener); + state.changeSubscription(Arrays.asList("test", "test1")); + + state.unsubscribe(); + + assertEquals( + "Expected subscribed topics count is incorrect", 0, state.subscription().size()); + } + + private static class MockRebalanceListener implements ConsumerRebalanceListener { + 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/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java new file mode 100644 index 0000000000000..d1759ceac5074 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -0,0 +1,76 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.producer; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.config.SSLConfigs; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.test.MockMetricsReporter; +import org.apache.kafka.test.MockSerializer; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Properties; +import java.util.Map; +import java.util.HashMap; + +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() throws Exception { + Map configs = new HashMap(); + configs.put(ProducerConfig.CLIENT_ID_CONFIG, "testConstructorClose"); + configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); + configs.put(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG, MockMetricsReporter.class.getName()); + configs.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL); + configs.put(SSLConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, Class.forName(SSLConfigs.DEFAULT_PRINCIPAL_BUILDER_CLASS)); + final int oldInitCount = MockSerializer.INIT_COUNT.get(); + final int oldCloseCount = MockSerializer.CLOSE_COUNT.get(); + + KafkaProducer producer = new KafkaProducer( + configs, 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 4547bfcb44be4..0000000000000 --- a/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java +++ /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. - */ -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.common.errors.TimeoutException; -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.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); - } - - 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(TimeoutException e) { - // let it go - } - } - } - }; - 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 f06e28ce21e80..0000000000000 --- a/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java +++ /dev/null @@ -1,81 +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.assertNotSame; -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 testRoundRobinIsStable() { - int startPart = partitioner.partition(new ProducerRecord("test", value), cluster); - for (int i = 1; i <= 100; i++) { - int partition = partitioner.partition(new ProducerRecord("test", value), cluster); - assertEquals("Should yield a different partition each call with round-robin partitioner", - partition, (startPart + i) % 2); - } - } - - @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 0762b35abba05..0000000000000 --- a/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java +++ /dev/null @@ -1,162 +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.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.Set; - -import org.apache.kafka.clients.producer.internals.RecordAccumulator; -import org.apache.kafka.clients.producer.internals.RecordBatch; -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 Node node = new Node(0, "localhost", 1111); - private TopicPartition tp1 = new TopicPartition(topic, partition1); - private TopicPartition tp2 = new TopicPartition(topic, partition2); - private PartitionInfo part1 = new PartitionInfo(topic, partition1, node, null, null); - private PartitionInfo part2 = new PartitionInfo(topic, partition2, node, 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(Collections.singleton(node), Arrays.asList(part1, part2)); - private Metrics metrics = new Metrics(time); - - @Test - public void testFull() throws Exception { - long now = time.milliseconds(); - RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, 10L, 100L, false, metrics, time); - int appends = 1024 / msgSize; - for (int i = 0; i < appends; i++) { - accum.append(tp1, key, value, CompressionType.NONE, null); - assertEquals("No partitions should be ready.", 0, accum.ready(cluster, now).readyNodes.size()); - } - accum.append(tp1, key, value, CompressionType.NONE, null); - assertEquals("Our partition's leader should be ready", Collections.singleton(node), accum.ready(cluster, time.milliseconds()).readyNodes); - List batches = accum.drain(cluster, Collections.singleton(node), Integer.MAX_VALUE, 0).get(node.id()); - 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, 100L, false, metrics, time); - accum.append(tp1, key, new byte[2 * batchSize], CompressionType.NONE, null); - assertEquals("Our partition's leader should be ready", Collections.singleton(node), accum.ready(cluster, time.milliseconds()).readyNodes); - } - - @Test - public void testLinger() throws Exception { - long lingerMs = 10L; - RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, lingerMs, 100L, false, metrics, time); - accum.append(tp1, key, value, CompressionType.NONE, 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(node), accum.ready(cluster, time.milliseconds()).readyNodes); - List batches = accum.drain(cluster, Collections.singleton(node), Integer.MAX_VALUE, 0).get(node.id()); - 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, 100L, false, metrics, time); - 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, CompressionType.NONE, null); - } - assertEquals("Partition's leader should be ready", Collections.singleton(node), accum.ready(cluster, time.milliseconds()).readyNodes); - - List batches = accum.drain(cluster, Collections.singleton(node), 1024, 0).get(node.id()); - 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 = 2; - final RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, 0L, 100L, 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(topic, 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) { - Set nodes = accum.ready(cluster, now).readyNodes; - List batches = accum.drain(cluster, nodes, 5 * 1024, 0).get(node.id()); - if (batches != null) { - for (RecordBatch batch : batches) { - for (LogEntry entry : batch.records) - read++; - accum.deallocate(batch); - } - } - } - - 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 ef2ca65cabe97..0000000000000 --- a/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java +++ /dev/null @@ -1,150 +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 static org.junit.Assert.fail; - -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; - -import org.apache.kafka.clients.MockClient; -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.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.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); - private RecordAccumulator accumulator = new RecordAccumulator(batchSize, 1024 * 1024, 0L, 0L, false, metrics, time); - private Sender sender = new Sender(client, - metadata, - this.accumulator, - MAX_REQUEST_SIZE, - ACKS_ALL, - MAX_RETRIES, - REQUEST_TIMEOUT_MS, - metrics, - time); - - @Before - public void setup() { - metadata.update(cluster, time.milliseconds()); - } - - @Test - public void testSimple() throws Exception { - int offset = 0; - Future future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, 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.topic(), tp.partition(), offset, Errors.NONE.code())); - sender.run(time.milliseconds()); - assertEquals("All requests completed.", offset, 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); - // do a successful retry - Future future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, 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()); - int offset = 0; - client.respond(produceResponse(tp.topic(), tp.partition(), 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(), CompressionType.NONE, 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(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 }); - return struct; - } - -} 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 72% 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 fe3c13f319d48..f8567e9f3a0a7 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,17 +14,20 @@ * 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 org.apache.kafka.clients.producer.internals.BufferPool; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.test.TestUtils; +import org.junit.After; 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; @@ -33,16 +36,24 @@ public class BufferPoolTest { private MockTime time = new MockTime(); private Metrics metrics = new Metrics(time); + private final long maxBlockTimeMs = 2000; + String metricGroup = "TestMetrics"; + Map metricTags = new LinkedHashMap(); + + @After + public void teardown() { + this.metrics.close(); + } /** * 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, metrics, time); - ByteBuffer buffer = pool.allocate(size); + BufferPool pool = new BufferPool(totalMemory, size, metrics, time, metricGroup, metricTags); + ByteBuffer buffer = pool.allocate(size, maxBlockTimeMs); assertEquals("Buffer size should equal requested size.", size, buffer.limit()); assertEquals("Unallocated memory should have shrunk", totalMemory - size, pool.unallocatedMemory()); assertEquals("Available memory should have shrunk", totalMemory - size, pool.availableMemory()); @@ -51,13 +62,13 @@ public void testSimple() throws Exception { pool.deallocate(buffer); assertEquals("All memory should be available", totalMemory, pool.availableMemory()); assertEquals("But now some is on the free list", totalMemory - size, pool.unallocatedMemory()); - buffer = pool.allocate(size); + buffer = pool.allocate(size, maxBlockTimeMs); assertEquals("Recycled buffer should be cleared.", 0, buffer.position()); assertEquals("Recycled buffer should be cleared.", buffer.capacity(), buffer.limit()); pool.deallocate(buffer); assertEquals("All memory should be available", totalMemory, pool.availableMemory()); assertEquals("Still a single buffer on the free list", totalMemory - size, pool.unallocatedMemory()); - buffer = pool.allocate(2 * size); + buffer = pool.allocate(2 * size, maxBlockTimeMs); pool.deallocate(buffer); assertEquals("All memory should be available", totalMemory, pool.availableMemory()); assertEquals("Non-standard size didn't go to the free list.", totalMemory - size, pool.unallocatedMemory()); @@ -68,23 +79,11 @@ public void testSimple() throws Exception { */ @Test(expected = IllegalArgumentException.class) public void testCantAllocateMoreMemoryThanWeHave() throws Exception { - BufferPool pool = new BufferPool(1024, 512, true, metrics, time); - ByteBuffer buffer = pool.allocate(1024); + BufferPool pool = new BufferPool(1024, 512, metrics, time, metricGroup, metricTags); + ByteBuffer buffer = pool.allocate(1024, maxBlockTimeMs); assertEquals(1024, buffer.limit()); pool.deallocate(buffer); - buffer = pool.allocate(1025); - } - - @Test - public void testNonblockingMode() throws Exception { - BufferPool pool = new BufferPool(2, 1, false, metrics, time); - pool.allocate(1); - try { - pool.allocate(2); - fail("The buffer allocated more than it has!"); - } catch (BufferExhaustedException e) { - // this is good - } + buffer = pool.allocate(1025, maxBlockTimeMs); } /** @@ -92,18 +91,18 @@ public void testNonblockingMode() throws Exception { */ @Test public void testDelayedAllocation() throws Exception { - BufferPool pool = new BufferPool(5 * 1024, 1024, true, metrics, time); - ByteBuffer buffer = pool.allocate(1024); + BufferPool pool = new BufferPool(5 * 1024, 1024, metrics, time, metricGroup, metricTags); + ByteBuffer buffer = pool.allocate(1024, maxBlockTimeMs); 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,26 +111,45 @@ 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); + pool.allocate(size, maxBlockTimeMs); } catch (InterruptedException e) { e.printStackTrace(); } finally { completed.countDown(); } } - }.start(); + }; + thread.start(); return completed; } + /** + * Test if Timeout exception is thrown when there is not enough memory to allocate and the elapsed time is greater than the max specified block time + * + * @throws Exception + */ + @Test + public void testBlockTimeout() throws Exception { + BufferPool pool = new BufferPool(2, 1, metrics, time, metricGroup, metricTags); + pool.allocate(1, maxBlockTimeMs); + try { + pool.allocate(2, maxBlockTimeMs); + fail("The buffer allocated more memory than its maximum value 2"); + } catch (TimeoutException e) { + // this is good + } + } + /** * This test creates lots of threads that hammer on the pool */ @@ -140,8 +158,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, metrics, time); + final long totalMemory = numThreads / 2 * poolableSize; + final BufferPool pool = new BufferPool(totalMemory, poolableSize, metrics, time, metricGroup, metricTags); List threads = new ArrayList(); for (int i = 0; i < numThreads; i++) threads.add(new StressTestThread(pool, iterations)); @@ -157,6 +175,7 @@ public void testStressfulSituation() throws Exception { public static class StressTestThread extends Thread { private final int iterations; private final BufferPool pool; + private final long maxBlockTimeMs = 2000; public final AtomicBoolean success = new AtomicBoolean(false); public StressTestThread(BufferPool pool, int iterations) { @@ -168,13 +187,13 @@ 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()); - ByteBuffer buffer = pool.allocate(size); + size = TestUtils.RANDOM.nextInt((int) pool.totalMemory()); + ByteBuffer buffer = pool.allocate(size, maxBlockTimeMs); pool.deallocate(buffer); } success.set(true); 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..887499dbba8e4 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java @@ -0,0 +1,323 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES 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.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.common.utils.Time; +import org.junit.After; +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(); + private final long maxBlockTimeMs = 1000; + + @After + public void teardown() { + this.metrics.close(); + } + + @Test + public void testFull() throws Exception { + long now = time.milliseconds(); + RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 10L, 100L, metrics, time, metricTags); + int appends = 1024 / msgSize; + for (int i = 0; i < appends; i++) { + accum.append(tp1, key, value, null, maxBlockTimeMs); + assertEquals("No partitions should be ready.", 0, accum.ready(cluster, now).readyNodes.size()); + } + accum.append(tp1, key, value, null, maxBlockTimeMs); + 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, metrics, time, metricTags); + accum.append(tp1, key, new byte[2 * batchSize], null, maxBlockTimeMs); + 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, metrics, time, metricTags); + accum.append(tp1, key, value, null, maxBlockTimeMs); + 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, 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, maxBlockTimeMs); + } + 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, 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, maxBlockTimeMs); + } 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, 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, maxBlockTimeMs); + 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, maxBlockTimeMs); + 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, maxBlockTimeMs); + 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, metrics, time, metricTags); + + long now = time.milliseconds(); + accum.append(tp1, key, value, null, maxBlockTimeMs); + 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, maxBlockTimeMs); + 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, metrics, time, metricTags); + for (int i = 0; i < 100; i++) + accum.append(new TopicPartition(topic, i % 3), key, value, null, maxBlockTimeMs); + 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, 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(), maxBlockTimeMs); + 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()); + + } + + @Test + public void testExpiredBatches() throws InterruptedException { + Time time = new SystemTime(); + long now = time.milliseconds(); + RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 10, 100L, metrics, time, metricTags); + int appends = 1024 / msgSize; + for (int i = 0; i < appends; i++) { + accum.append(tp1, key, value, null, maxBlockTimeMs); + assertEquals("No partitions should be ready.", 0, accum.ready(cluster, now).readyNodes.size()); + } + time.sleep(2000); + accum.ready(cluster, now); + accum.append(tp1, key, value, null, 0); + Set readyNodes = accum.ready(cluster, time.milliseconds()).readyNodes; + assertEquals("Our partition's leader should be ready", Collections.singleton(node1), readyNodes); + Cluster cluster = new Cluster(new ArrayList(), new ArrayList()); + now = time.milliseconds(); + List expiredBatches = accum.abortExpiredBatches(60, cluster, now); + assertEquals(1, expiredBatches.size()); + } +} 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..bcc618aad69b7 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.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.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.MetricName; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.metrics.KafkaMetric; +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.After; +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 String CLIENT_ID = "clientId"; + private static final String METRIC_GROUP = "producer-metrics"; + private static final double EPS = 0.0001; + private static final int MAX_BLOCK_TIMEOUT = 1000; + private static final int REQUEST_TIMEOUT = 1000; + + 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, metrics, time, metricTags); + private Sender sender = new Sender(client, + metadata, + this.accumulator, + MAX_REQUEST_SIZE, + ACKS_ALL, + MAX_RETRIES, + metrics, + time, + CLIENT_ID, + REQUEST_TIMEOUT); + + @Before + public void setup() { + metadata.update(cluster, time.milliseconds()); + metricTags.put("client-id", CLIENT_ID); + } + + @After + public void tearDown() { + this.metrics.close(); + } + + @Test + public void testSimple() throws Exception { + long offset = 0; + Future future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), null, MAX_BLOCK_TIMEOUT).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(), 0)); + 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()); + } + + /* + * Send multiple requests. Verify that the client side quota metrics have the right values + */ + @Test + public void testQuotaMetrics() throws Exception { + final long offset = 0; + for (int i = 1; i <= 3; i++) { + Future future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), null, MAX_BLOCK_TIMEOUT).future; + sender.run(time.milliseconds()); // send produce request + client.respond(produceResponse(tp, offset, Errors.NONE.code(), 100 * i)); + sender.run(time.milliseconds()); + } + Map allMetrics = metrics.metrics(); + KafkaMetric avgMetric = allMetrics.get(new MetricName("produce-throttle-time-avg", METRIC_GROUP, "", metricTags)); + KafkaMetric maxMetric = allMetrics.get(new MetricName("produce-throttle-time-max", METRIC_GROUP, "", metricTags)); + assertEquals(200, avgMetric.value(), EPS); + assertEquals(300, maxMetric.value(), EPS); + } + + @Test + public void testRetries() throws Exception { + // create a sender with retries = 1 + int maxRetries = 1; + Metrics m = new Metrics(); + try { + Sender sender = new Sender(client, + metadata, + this.accumulator, + MAX_REQUEST_SIZE, + ACKS_ALL, + maxRetries, + m, + time, + "clientId", + REQUEST_TIMEOUT); + // do a successful retry + Future future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), null, MAX_BLOCK_TIMEOUT).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(), 0)); + 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, MAX_BLOCK_TIMEOUT).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); + } finally { + m.close(); + } + } + + 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, int throttleTimeMs) { + ProduceResponse.PartitionResponse resp = new ProduceResponse.PartitionResponse((short) error, offset); + Map partResp = Collections.singletonMap(tp, resp); + ProduceResponse response = new ProduceResponse(partResp, throttleTimeMs); + return response.toStruct(); + } + +} diff --git a/clients/src/test/java/org/apache/kafka/common/cache/LRUCacheTest.java b/clients/src/test/java/org/apache/kafka/common/cache/LRUCacheTest.java new file mode 100644 index 0000000000000..4cf130ce09043 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/cache/LRUCacheTest.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.common.cache; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +public class LRUCacheTest { + + @Test + public void testPutGet() { + Cache cache = new LRUCache<>(4); + + cache.put("a", "b"); + cache.put("c", "d"); + cache.put("e", "f"); + cache.put("g", "h"); + + assertEquals(4, cache.size()); + + assertEquals("b", cache.get("a")); + assertEquals("d", cache.get("c")); + assertEquals("f", cache.get("e")); + assertEquals("h", cache.get("g")); + } + + @Test + public void testRemove() { + Cache cache = new LRUCache<>(4); + + cache.put("a", "b"); + cache.put("c", "d"); + cache.put("e", "f"); + assertEquals(3, cache.size()); + + assertEquals(true, cache.remove("a")); + assertEquals(2, cache.size()); + assertNull(cache.get("a")); + assertEquals("d", cache.get("c")); + assertEquals("f", cache.get("e")); + + assertEquals(false, cache.remove("key-does-not-exist")); + + assertEquals(true, cache.remove("c")); + assertEquals(1, cache.size()); + assertNull(cache.get("c")); + assertEquals("f", cache.get("e")); + + assertEquals(true, cache.remove("e")); + assertEquals(0, cache.size()); + assertNull(cache.get("e")); + } + + @Test + public void testEviction() { + Cache cache = new LRUCache<>(2); + + cache.put("a", "b"); + cache.put("c", "d"); + assertEquals(2, cache.size()); + + cache.put("e", "f"); + assertEquals(2, cache.size()); + assertNull(cache.get("a")); + assertEquals("d", cache.get("c")); + assertEquals("f", cache.get("e")); + + // Validate correct access order eviction + cache.get("c"); + cache.put("g", "h"); + assertEquals(2, cache.size()); + assertNull(cache.get("e")); + assertEquals("d", cache.get("c")); + assertEquals("h", cache.get("g")); + } +} 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..28064ec8496a5 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.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.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.HashMap; +import java.util.Map; +import java.util.Properties; + +import static org.junit.Assert.fail; +import static org.junit.Assert.assertEquals; + +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,"); + } + + @Test + public void testOriginalsWithPrefix() { + Properties props = new Properties(); + props.put("foo.bar", "abc"); + props.put("setting", "def"); + TestConfig config = new TestConfig(props); + Map expected = new HashMap<>(); + expected.put("bar", "abc"); + assertEquals(expected, config.originalsWithPrefix("foo.")); + } + + 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 09a82feeb7cae..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 @@ -21,7 +21,9 @@ import java.util.Properties; 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,7 +37,9 @@ public void testBasicTypes() { .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("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 "); @@ -44,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")); @@ -53,6 +59,8 @@ 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) @@ -82,6 +90,7 @@ 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) { @@ -97,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..d5dd9b8b5631a --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/metrics/FakeMetricsReporter.java @@ -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 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 metricRemoval(KafkaMetric metric) {} + + @Override + public void close() {} + +} 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..90cd76f923340 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; @@ -29,12 +26,16 @@ public class JmxReporterTest { @Test 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 sensor2 = metrics.sensor("kafka.blah"); - sensor2.add("pack.bean1.some", new Total()); - sensor2.add("pack.bean2.some", new Total()); + try { + metrics.addReporter(new JmxReporter()); + Sensor sensor = metrics.sensor("kafka.requests"); + 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(new MetricName("pack.bean1.some", "grp1"), new Total()); + sensor2.add(new MetricName("pack.bean2.some", "grp1"), new Total()); + } finally { + metrics.close(); + } } } 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 19bea0f1fa1eb..90583870a079e 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 @@ -13,12 +13,19 @@ package org.apache.kafka.common.metrics; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; 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.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; @@ -29,60 +36,105 @@ import org.apache.kafka.common.metrics.stats.Rate; import org.apache.kafka.common.metrics.stats.Total; import org.apache.kafka.common.utils.MockTime; +import org.junit.After; +import org.junit.Before; import org.junit.Test; public class MetricsTest { - private static double EPS = 0.000001; + private static final double EPS = 0.000001; + private MockTime time = new MockTime(); + private MetricConfig config = new MetricConfig(); + private Metrics metrics; - MockTime time = new MockTime(); - Metrics metrics = new Metrics(new MetricConfig(), Arrays.asList((MetricsReporter) new JmxReporter()), time); + @Before + public void setup() { + this.metrics = new Metrics(config, Arrays.asList((MetricsReporter) new JmxReporter()), time); + } + + @After + public void tearDown() { + this.metrics.close(); + } + + @Test + public void testMetricName() { + MetricName n1 = new MetricName("name", "group", "description", "key1", "value1", "key2", "value2"); + Map tags = new HashMap(); + tags.put("key1", "value1"); + tags.put("key2", "value2"); + 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++) + int sum = 0; + int count = 10; + for (int i = 0; i < count; i++) { s.record(i); + sum += i; + } + // prior to any time passing + double elapsedSecs = (config.timeWindowMs() * (config.samples() - 1)) / 1000.0; + assertEquals(String.format("Occurrences(0...%d) = %f", count, count / elapsedSecs), count / elapsedSecs, + metrics.metrics().get(new MetricName("test.occurences", "grp1")).value(), EPS); // pretend 2 seconds passed... - time.sleep(2000); + long sleepTimeMs = 2; + time.sleep(sleepTimeMs * 1000); + elapsedSecs += sleepTimeMs; - 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", count - 1, 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) = 1.40625", + sum / elapsedSecs, metrics.metrics().get(new MetricName("test.rate", "grp1")).value(), EPS); + assertEquals(String.format("Occurrences(0...%d) = %f", count, count / elapsedSecs), + count / elapsedSecs, + metrics.metrics().get(new MetricName("test.occurences", "grp1")).value(), EPS); + assertEquals("Count(0...9) = 10", + (double) count, 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(); @@ -99,20 +151,131 @@ public void testHierarchicalSensors() { /* each metric should have a count equal to one + its children's count */ assertEquals(1.0, gc, EPS); - assertEquals(1.0 + gc, child1.metrics().get(0).value(), EPS); + assertEquals(1.0 + gc, c1, EPS); assertEquals(1.0, c2, EPS); assertEquals(1.0 + c1, p2, EPS); assertEquals(1.0 + c1 + c2, p1, EPS); } @Test(expected = IllegalArgumentException.class) - public void testBadSensorHiearchy() { + public void testBadSensorHierarchy() { Sensor p = metrics.sensor("parent"); Sensor c1 = metrics.sensor("child1", p); Sensor c2 = metrics.sensor("child2", p); metrics.sensor("gc", c1, c2); // should fail } + @Test + public void testRemoveSensor() { + Sensor parent1 = metrics.sensor("test.parent1"); + parent1.add(new MetricName("test.parent1.count", "grp1"), new Count()); + Sensor parent2 = metrics.sensor("test.parent2"); + parent2.add(new MetricName("test.parent2.count", "grp1"), new Count()); + Sensor child1 = metrics.sensor("test.child1", parent1, parent2); + child1.add(new MetricName("test.child1.count", "grp1"), new Count()); + Sensor child2 = metrics.sensor("test.child2", parent2); + child2.add(new MetricName("test.child2.count", "grp1"), new Count()); + Sensor grandChild1 = metrics.sensor("test.gchild2", child2); + grandChild1.add(new MetricName("test.gchild2.count", "grp1"), new Count()); + + Sensor sensor = metrics.getSensor("test.parent1"); + assertNotNull(sensor); + metrics.removeSensor("test.parent1"); + assertNull(metrics.getSensor("test.parent1")); + assertNull(metrics.metrics().get(new MetricName("test.parent1.count", "grp1"))); + assertNull(metrics.getSensor("test.child1")); + assertNull(metrics.childrenSensors().get(sensor)); + assertNull(metrics.metrics().get(new MetricName("test.child1.count", "grp1"))); + + sensor = metrics.getSensor("test.gchild2"); + assertNotNull(sensor); + metrics.removeSensor("test.gchild2"); + assertNull(metrics.getSensor("test.gchild2")); + assertNull(metrics.childrenSensors().get(sensor)); + assertNull(metrics.metrics().get(new MetricName("test.gchild2.count", "grp1"))); + + sensor = metrics.getSensor("test.child2"); + assertNotNull(sensor); + metrics.removeSensor("test.child2"); + assertNull(metrics.getSensor("test.child2")); + assertNull(metrics.childrenSensors().get(sensor)); + assertNull(metrics.metrics().get(new MetricName("test.child2.count", "grp1"))); + + sensor = metrics.getSensor("test.parent2"); + assertNotNull(sensor); + metrics.removeSensor("test.parent2"); + assertNull(metrics.getSensor("test.parent2")); + assertNull(metrics.childrenSensors().get(sensor)); + assertNull(metrics.metrics().get(new MetricName("test.parent2.count", "grp1"))); + + assertEquals(0, metrics.metrics().size()); + } + + @Test + public void testRemoveInactiveMetrics() { + Sensor s1 = metrics.sensor("test.s1", null, 1); + s1.add(new MetricName("test.s1.count", "grp1"), new Count()); + + Sensor s2 = metrics.sensor("test.s2", null, 3); + s2.add(new MetricName("test.s2.count", "grp1"), new Count()); + + Metrics.ExpireSensorTask purger = metrics.new ExpireSensorTask(); + purger.run(); + assertNotNull("Sensor test.s1 must be present", metrics.getSensor("test.s1")); + assertNotNull("MetricName test.s1.count must be present", + metrics.metrics().get(new MetricName("test.s1.count", "grp1"))); + assertNotNull("Sensor test.s2 must be present", metrics.getSensor("test.s2")); + assertNotNull("MetricName test.s2.count must be present", + metrics.metrics().get(new MetricName("test.s2.count", "grp1"))); + + time.sleep(1001); + purger.run(); + assertNull("Sensor test.s1 should have been purged", metrics.getSensor("test.s1")); + assertNull("MetricName test.s1.count should have been purged", + metrics.metrics().get(new MetricName("test.s1.count", "grp1"))); + assertNotNull("Sensor test.s2 must be present", metrics.getSensor("test.s2")); + assertNotNull("MetricName test.s2.count must be present", + metrics.metrics().get(new MetricName("test.s2.count", "grp1"))); + + // record a value in sensor s2. This should reset the clock for that sensor. + // It should not get purged at the 3 second mark after creation + s2.record(); + time.sleep(2000); + purger.run(); + assertNotNull("Sensor test.s2 must be present", metrics.getSensor("test.s2")); + assertNotNull("MetricName test.s2.count must be present", + metrics.metrics().get(new MetricName("test.s2.count", "grp1"))); + + // After another 1 second sleep, the metric should be purged + time.sleep(1000); + purger.run(); + assertNull("Sensor test.s2 should have been purged", metrics.getSensor("test.s1")); + assertNull("MetricName test.s2.count should have been purged", + metrics.metrics().get(new MetricName("test.s1.count", "grp1"))); + + // After purging, it should be possible to recreate a metric + s1 = metrics.sensor("test.s1", null, 1); + s1.add(new MetricName("test.s1.count", "grp1"), new Count()); + assertNotNull("Sensor test.s1 must be present", metrics.getSensor("test.s1")); + assertNotNull("MetricName test.s1.count must be present", + metrics.metrics().get(new MetricName("test.s1.count", "grp1"))); + } + + @Test + public void testRemoveMetric() { + metrics.addMetric(new MetricName("test1", "grp1"), new Count()); + metrics.addMetric(new MetricName("test2", "grp1"), new Count()); + + assertNotNull(metrics.removeMetric(new MetricName("test1", "grp1"))); + assertNull(metrics.metrics().get(new MetricName("test1", "grp1"))); + assertNotNull(metrics.metrics().get(new MetricName("test2", "grp1"))); + + assertNotNull(metrics.removeMetric(new MetricName("test2", "grp1"))); + assertNull(metrics.metrics().get(new MetricName("test2", "grp1"))); + + assertEquals(0, metrics.metrics().size()); + } + @Test public void testEventWindowing() { Count count = new Count(); @@ -150,15 +313,15 @@ public void testOldDataHasNoEffect() { @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); @@ -166,7 +329,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); @@ -176,6 +339,18 @@ public void testQuotas() { } } + @Test + public void testQuotasEquality() { + final Quota quota1 = Quota.lessThan(10.5); + final Quota quota2 = Quota.moreThan(10.5); + + assertFalse("Quota with different upper values shouldn't be equal", quota1.equals(quota2)); + + final Quota quota3 = Quota.moreThan(10.5); + + assertTrue("Quota with same upper and bound values should be equal", quota2.equals(quota3)); + } + @Test public void testPercentiles() { int buckets = 100; @@ -183,15 +358,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++) @@ -209,6 +384,34 @@ public void testPercentiles() { assertEquals(0.0, p75.value(), 1.0); } + @Test + public void testRateWindowing() throws Exception { + // Use the default time window. Set 3 samples + MetricConfig cfg = new MetricConfig().samples(3); + Sensor s = metrics.sensor("test.sensor", cfg); + s.add(new MetricName("test.rate", "grp1"), new Rate(TimeUnit.SECONDS)); + + int sum = 0; + int count = cfg.samples() - 1; + // Advance 1 window after every record + for (int i = 0; i < count; i++) { + s.record(100); + sum += 100; + time.sleep(cfg.timeWindowMs()); + } + + // Sleep for half the window. + time.sleep(cfg.timeWindowMs() / 2); + + // prior to any time passing + double elapsedSecs = (cfg.timeWindowMs() * (cfg.samples() - 1) + cfg.timeWindowMs() / 2) / 1000.0; + + KafkaMetric km = metrics.metrics().get(new MetricName("test.rate", "grp1")); + assertEquals("Rate(0...2) = 2.666", sum / elapsedSecs, km.value(), EPS); + assertEquals("Elapsed Time = 75 seconds", elapsedSecs, + ((Rate) km.measurable()).windowSize(cfg, time.milliseconds()) / 1000, EPS); + } + public static class ConstantMeasurable implements Measurable { public double value = 0.0; 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/EchoServer.java b/clients/src/test/java/org/apache/kafka/common/network/EchoServer.java new file mode 100644 index 0000000000000..f13c21ab94317 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/network/EchoServer.java @@ -0,0 +1,119 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES 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.protocol.SecurityProtocol; +import org.apache.kafka.common.security.ssl.SSLFactory; + +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLSocket; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.net.ServerSocket; +import java.net.Socket; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Map; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; + + +/** + * A simple server that takes size delimited byte arrays and just echos them back to the sender. + */ +class EchoServer extends Thread { + public final int port; + private final ServerSocket serverSocket; + private final List threads; + private final List sockets; + private SecurityProtocol protocol = SecurityProtocol.PLAINTEXT; + private SSLFactory sslFactory; + private final AtomicBoolean renegotiate = new AtomicBoolean(); + + public EchoServer(Map configs) throws Exception { + this.protocol = configs.containsKey("security.protocol") ? + SecurityProtocol.valueOf((String) configs.get("security.protocol")) : SecurityProtocol.PLAINTEXT; + if (protocol == SecurityProtocol.SSL) { + this.sslFactory = new SSLFactory(SSLFactory.Mode.SERVER); + this.sslFactory.configure(configs); + SSLContext sslContext = this.sslFactory.sslContext(); + this.serverSocket = sslContext.getServerSocketFactory().createServerSocket(0); + } else { + this.serverSocket = new ServerSocket(0); + } + this.port = this.serverSocket.getLocalPort(); + this.threads = Collections.synchronizedList(new ArrayList()); + this.sockets = Collections.synchronizedList(new ArrayList()); + } + + public void renegotiate() { + renegotiate.set(true); + } + + @Override + public void run() { + try { + while (true) { + final Socket socket = serverSocket.accept(); + sockets.add(socket); + Thread thread = new Thread() { + @Override + public void run() { + try { + DataInputStream input = new DataInputStream(socket.getInputStream()); + DataOutputStream output = new DataOutputStream(socket.getOutputStream()); + while (socket.isConnected() && !socket.isClosed()) { + int size = input.readInt(); + if (renegotiate.get()) { + renegotiate.set(false); + ((SSLSocket) socket).startHandshake(); + } + byte[] bytes = new byte[size]; + input.readFully(bytes); + output.writeInt(size); + output.write(bytes); + output.flush(); + } + } catch (IOException e) { + // ignore + } finally { + try { + socket.close(); + } catch (IOException e) { + // ignore + } + } + } + }; + thread.start(); + threads.add(thread); + } + } catch (IOException e) { + // ignore + } + } + + public void closeConnections() throws IOException { + for (Socket socket : sockets) + socket.close(); + } + + public void close() throws IOException, InterruptedException { + this.serverSocket.close(); + closeConnections(); + for (Thread t : threads) + t.join(); + join(); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/network/SSLSelectorTest.java b/clients/src/test/java/org/apache/kafka/common/network/SSLSelectorTest.java new file mode 100644 index 0000000000000..c60053fd24ca0 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/network/SSLSelectorTest.java @@ -0,0 +1,121 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.common.network; + +import static org.junit.Assert.assertEquals; + +import java.util.LinkedHashMap; +import java.util.Map; +import java.io.File; +import java.io.IOException; +import java.net.InetSocketAddress; + +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.config.SSLConfigs; +import org.apache.kafka.common.security.ssl.SSLFactory; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.test.TestSSLUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +/** + * A set of tests for the selector. These use a test harness that runs a simple socket server that echos back responses. + */ +public class SSLSelectorTest extends SelectorTest { + + private Metrics metrics; + + @Before + public void setup() throws Exception { + File trustStoreFile = File.createTempFile("truststore", ".jks"); + + Map sslServerConfigs = TestSSLUtils.createSSLConfig(false, true, SSLFactory.Mode.SERVER, trustStoreFile, "server"); + sslServerConfigs.put(SSLConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, Class.forName(SSLConfigs.DEFAULT_PRINCIPAL_BUILDER_CLASS)); + this.server = new EchoServer(sslServerConfigs); + this.server.start(); + this.time = new MockTime(); + Map sslClientConfigs = TestSSLUtils.createSSLConfig(false, false, SSLFactory.Mode.SERVER, trustStoreFile, "client"); + sslClientConfigs.put(SSLConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, Class.forName(SSLConfigs.DEFAULT_PRINCIPAL_BUILDER_CLASS)); + + this.channelBuilder = new SSLChannelBuilder(SSLFactory.Mode.CLIENT); + this.channelBuilder.configure(sslClientConfigs); + this.metrics = new Metrics(); + this.selector = new Selector(5000, metrics, time, "MetricGroup", new LinkedHashMap(), channelBuilder); + } + + @After + public void teardown() throws Exception { + this.selector.close(); + this.server.close(); + this.metrics.close(); + } + + /** + * Tests that SSL renegotiation initiated by the server are handled correctly by the client + * @throws Exception + */ + @Test + public void testRenegotiation() throws Exception { + int reqs = 500; + String node = "0"; + // create connections + InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); + + // send echo requests and receive responses + int requests = 0; + int responses = 0; + int renegotiates = 0; + while (!selector.isChannelReady(node)) { + selector.poll(1000L); + } + selector.send(createSend(node, node + "-" + 0)); + requests++; + + // loop until we complete all requests + while (responses < reqs) { + selector.poll(0L); + if (responses >= 100 && renegotiates == 0) { + renegotiates++; + server.renegotiate(); + } + assertEquals("No disconnects should have occurred.", 0, selector.disconnected().size()); + + // handle any responses we may have gotten + 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(), pieces[0]); + assertEquals("Check that the receive has kindly been rewound", 0, receive.payload().position()); + assertEquals("Check the request counter", responses, Integer.parseInt(pieces[1])); + responses++; + } + + // prepare new sends for the next round + for (int i = 0; i < selector.completedSends().size() && requests < reqs && selector.isChannelReady(node); i++, requests++) { + selector.send(createSend(node, node + "-" + requests)); + } + } + } + + /** + * Connects and waits for handshake to complete. This is required since SSLTransportLayer + * implementation requires the channel to be ready before send is invoked (unlike plaintext + * where send can be invoked straight after connect) + */ + protected void connect(String node, InetSocketAddress serverAddr) throws IOException { + blockingConnect(node, serverAddr); + } + +} diff --git a/clients/src/test/java/org/apache/kafka/common/network/SSLTransportLayerTest.java b/clients/src/test/java/org/apache/kafka/common/network/SSLTransportLayerTest.java new file mode 100644 index 0000000000000..6993f5242bd1c --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/network/SSLTransportLayerTest.java @@ -0,0 +1,652 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES 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 org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.io.IOException; +import java.io.File; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.nio.ByteBuffer; +import java.nio.channels.SelectionKey; +import java.nio.channels.ServerSocketChannel; +import java.nio.channels.SocketChannel; + +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLEngine; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.config.SSLConfigs; +import org.apache.kafka.common.security.ssl.SSLFactory; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.test.TestSSLUtils; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +/** + * Tests for the SSL transport layer. These use a test harness that runs a simple socket server that echos back responses. + */ + +public class SSLTransportLayerTest { + + private static final int BUFFER_SIZE = 4 * 1024; + + private SSLEchoServer server; + private Selector selector; + private ChannelBuilder channelBuilder; + private CertStores serverCertStores; + private CertStores clientCertStores; + private Map sslClientConfigs; + private Map sslServerConfigs; + + @Before + public void setup() throws Exception { + // Create certificates for use by client and server. Add server cert to client truststore and vice versa. + serverCertStores = new CertStores(true); + clientCertStores = new CertStores(false); + sslServerConfigs = serverCertStores.getTrustingConfig(clientCertStores); + sslClientConfigs = clientCertStores.getTrustingConfig(serverCertStores); + + this.channelBuilder = new SSLChannelBuilder(SSLFactory.Mode.CLIENT); + this.channelBuilder.configure(sslClientConfigs); + this.selector = new Selector(5000, new Metrics(), new MockTime(), "MetricGroup", new LinkedHashMap(), channelBuilder); + } + + @After + public void teardown() throws Exception { + if (selector != null) + this.selector.close(); + if (server != null) + this.server.close(); + } + + /** + * Tests that server certificate with valid IP address is accepted by + * a client that validates server endpoint. + */ + @Test + public void testValidEndpointIdentification() throws Exception { + String node = "0"; + createEchoServer(sslServerConfigs); + sslClientConfigs.put(SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, "HTTPS"); + createSelector(sslClientConfigs); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); + + testClientConnection(node, 100, 10); + } + + /** + * Tests that server certificate with invalid IP address is not accepted by + * a client that validates server endpoint. Certificate uses "localhost" as + * common name, test uses host IP to trigger endpoint validation failure. + */ + @Test + public void testInvalidEndpointIdentification() throws Exception { + String node = "0"; + String serverHost = InetAddress.getLocalHost().getHostAddress(); + server = new SSLEchoServer(sslServerConfigs, serverHost); + server.start(); + sslClientConfigs.put(SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, "HTTPS"); + createSelector(sslClientConfigs); + InetSocketAddress addr = new InetSocketAddress(serverHost, server.port); + selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); + + waitForChannelClose(node); + } + + /** + * Tests that server certificate with invalid IP address is accepted by + * a client that has disabled endpoint validation + */ + @Test + public void testEndpointIdentificationDisabled() throws Exception { + String node = "0"; + String serverHost = InetAddress.getLocalHost().getHostAddress(); + server = new SSLEchoServer(sslServerConfigs, serverHost); + server.start(); + sslClientConfigs.remove(SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG); + createSelector(sslClientConfigs); + InetSocketAddress addr = new InetSocketAddress(serverHost, server.port); + selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); + + testClientConnection(node, 100, 10); + } + + /** + * Tests that server accepts connections from clients with a trusted certificate + * when client authentication is required. + */ + @Test + public void testClientAuthenticationRequiredValidProvided() throws Exception { + String node = "0"; + sslServerConfigs.put(SSLConfigs.SSL_CLIENT_AUTH_CONFIG, "required"); + createEchoServer(sslServerConfigs); + createSelector(sslClientConfigs); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); + + testClientConnection(node, 100, 10); + } + + /** + * Tests that server does not accept connections from clients with an untrusted certificate + * when client authentication is required. + */ + @Test + public void testClientAuthenticationRequiredUntrustedProvided() throws Exception { + String node = "0"; + sslServerConfigs = serverCertStores.getUntrustingConfig(); + sslServerConfigs.put(SSLConfigs.SSL_CLIENT_AUTH_CONFIG, "required"); + createEchoServer(sslServerConfigs); + createSelector(sslClientConfigs); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); + + waitForChannelClose(node); + } + + /** + * Tests that server does not accept connections from clients which dont + * provide a certificate when client authentication is required. + */ + @Test + public void testClientAuthenticationRequiredNotProvided() throws Exception { + String node = "0"; + sslServerConfigs.put(SSLConfigs.SSL_CLIENT_AUTH_CONFIG, "required"); + createEchoServer(sslServerConfigs); + + sslClientConfigs.remove(SSLConfigs.SSL_KEYSTORE_LOCATION_CONFIG); + sslClientConfigs.remove(SSLConfigs.SSL_KEYSTORE_PASSWORD_CONFIG); + sslClientConfigs.remove(SSLConfigs.SSL_KEY_PASSWORD_CONFIG); + createSelector(sslClientConfigs); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); + + waitForChannelClose(node); + } + + /** + * Tests that server accepts connections from a client configured + * with an untrusted certificate if client authentication is disabled + */ + @Test + public void testClientAuthenticationDisabledUntrustedProvided() throws Exception { + String node = "0"; + sslServerConfigs = serverCertStores.getUntrustingConfig(); + sslServerConfigs.put(SSLConfigs.SSL_CLIENT_AUTH_CONFIG, "none"); + createEchoServer(sslServerConfigs); + createSelector(sslClientConfigs); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); + + testClientConnection(node, 100, 10); + } + + /** + * Tests that server accepts connections from a client that does not provide + * a certificate if client authentication is disabled + */ + @Test + public void testClientAuthenticationDisabledNotProvided() throws Exception { + String node = "0"; + sslServerConfigs.put(SSLConfigs.SSL_CLIENT_AUTH_CONFIG, "none"); + createEchoServer(sslServerConfigs); + + sslClientConfigs.remove(SSLConfigs.SSL_KEYSTORE_LOCATION_CONFIG); + sslClientConfigs.remove(SSLConfigs.SSL_KEYSTORE_PASSWORD_CONFIG); + sslClientConfigs.remove(SSLConfigs.SSL_KEY_PASSWORD_CONFIG); + createSelector(sslClientConfigs); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); + + testClientConnection(node, 100, 10); + } + + /** + * Tests that server accepts connections from a client configured + * with a valid certificate if client authentication is requested + */ + @Test + public void testClientAuthenticationRequestedValidProvided() throws Exception { + String node = "0"; + sslServerConfigs.put(SSLConfigs.SSL_CLIENT_AUTH_CONFIG, "requested"); + createEchoServer(sslServerConfigs); + createSelector(sslClientConfigs); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); + + testClientConnection(node, 100, 10); + } + + /** + * Tests that server accepts connections from a client that does not provide + * a certificate if client authentication is requested but not required + */ + @Test + public void testClientAuthenticationRequestedNotProvided() throws Exception { + String node = "0"; + sslServerConfigs.put(SSLConfigs.SSL_CLIENT_AUTH_CONFIG, "requested"); + createEchoServer(sslServerConfigs); + + sslClientConfigs.remove(SSLConfigs.SSL_KEYSTORE_LOCATION_CONFIG); + sslClientConfigs.remove(SSLConfigs.SSL_KEYSTORE_PASSWORD_CONFIG); + sslClientConfigs.remove(SSLConfigs.SSL_KEY_PASSWORD_CONFIG); + createSelector(sslClientConfigs); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); + + testClientConnection(node, 100, 10); + } + + /** + * Tests that channels cannot be created if truststore cannot be loaded + */ + @Test + public void testInvalidTruststorePassword() throws Exception { + SSLChannelBuilder channelBuilder = new SSLChannelBuilder(SSLFactory.Mode.CLIENT); + try { + sslClientConfigs.put(SSLConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "invalid"); + channelBuilder.configure(sslClientConfigs); + fail("SSL channel configured with invalid truststore password"); + } catch (KafkaException e) { + // Expected exception + } + } + + /** + * Tests that channels cannot be created if keystore cannot be loaded + */ + @Test + public void testInvalidKeystorePassword() throws Exception { + SSLChannelBuilder channelBuilder = new SSLChannelBuilder(SSLFactory.Mode.CLIENT); + try { + sslClientConfigs.put(SSLConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, "invalid"); + channelBuilder.configure(sslClientConfigs); + fail("SSL channel configured with invalid keystore password"); + } catch (KafkaException e) { + // Expected exception + } + } + + /** + * Tests that client connections cannot be created to a server + * if key password is invalid + */ + @Test + public void testInvalidKeyPassword() throws Exception { + String node = "0"; + sslServerConfigs.put(SSLConfigs.SSL_KEY_PASSWORD_CONFIG, "invalid"); + createEchoServer(sslServerConfigs); + createSelector(sslClientConfigs); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); + + waitForChannelClose(node); + } + + /** + * Tests that connections cannot be made with unsupported TLS versions + */ + @Test + public void testUnsupportedTLSVersion() throws Exception { + String node = "0"; + sslServerConfigs.put(SSLConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.2")); + createEchoServer(sslServerConfigs); + + sslClientConfigs.put(SSLConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.1")); + createSelector(sslClientConfigs); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); + + waitForChannelClose(node); + } + + /** + * Tests that connections cannot be made with unsupported TLS cipher suites + */ + @Test + public void testUnsupportedCiphers() throws Exception { + String node = "0"; + String[] cipherSuites = SSLContext.getDefault().getDefaultSSLParameters().getCipherSuites(); + sslServerConfigs.put(SSLConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuites[0])); + createEchoServer(sslServerConfigs); + + sslClientConfigs.put(SSLConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuites[1])); + createSelector(sslClientConfigs); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); + + waitForChannelClose(node); + } + + /** + * Tests handling of BUFFER_UNDERFLOW during unwrap when network read buffer is smaller than SSL session packet buffer size. + */ + @Test + public void testNetReadBufferResize() throws Exception { + String node = "0"; + createEchoServer(sslServerConfigs); + createSelector(sslClientConfigs, 10, null, null); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); + + testClientConnection(node, 64000, 10); + } + + /** + * Tests handling of BUFFER_OVERFLOW during wrap when network write buffer is smaller than SSL session packet buffer size. + */ + @Test + public void testNetWriteBufferResize() throws Exception { + String node = "0"; + createEchoServer(sslServerConfigs); + createSelector(sslClientConfigs, null, 10, null); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); + + testClientConnection(node, 64000, 10); + } + + /** + * Tests handling of BUFFER_OVERFLOW during unwrap when application read buffer is smaller than SSL session application buffer size. + */ + @Test + public void testApplicationBufferResize() throws Exception { + String node = "0"; + createEchoServer(sslServerConfigs); + createSelector(sslClientConfigs, null, null, 10); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); + + testClientConnection(node, 64000, 10); + } + + private void testClientConnection(String node, int minMessageSize, int messageCount) throws Exception { + + String prefix = TestUtils.randomString(minMessageSize); + int requests = 0; + int responses = 0; + // wait for handshake to finish + while (!selector.isChannelReady(node)) { + selector.poll(1000L); + } + selector.send(new NetworkSend(node, ByteBuffer.wrap((prefix + "-0").getBytes()))); + requests++; + while (responses < messageCount) { + selector.poll(0L); + assertEquals("No disconnects should have occurred.", 0, selector.disconnected().size()); + + for (NetworkReceive receive : selector.completedReceives()) { + assertEquals(prefix + "-" + responses, new String(Utils.toArray(receive.payload()))); + responses++; + } + + for (int i = 0; i < selector.completedSends().size() && requests < messageCount && selector.isChannelReady(node); i++, requests++) { + selector.send(new NetworkSend(node, ByteBuffer.wrap((prefix + "-" + requests).getBytes()))); + } + } + } + + private void waitForChannelClose(String node) throws IOException { + boolean closed = false; + for (int i = 0; i < 30; i++) { + selector.poll(1000L); + if (selector.channel(node) == null) { + closed = true; + break; + } + } + assertTrue(closed); + } + + private void createEchoServer(Map sslServerConfigs) throws Exception { + server = new SSLEchoServer(sslServerConfigs, "localhost"); + server.start(); + } + + private void createSelector(Map sslClientConfigs) { + createSelector(sslClientConfigs, null, null, null); + } + + private void createSelector(Map sslClientConfigs, final Integer netReadBufSize, final Integer netWriteBufSize, final Integer appBufSize) { + + this.channelBuilder = new SSLChannelBuilder(SSLFactory.Mode.CLIENT) { + + @Override + protected SSLTransportLayer buildTransportLayer(SSLFactory sslFactory, String id, SelectionKey key) throws IOException { + SocketChannel socketChannel = (SocketChannel) key.channel(); + SSLEngine sslEngine = sslFactory.createSSLEngine(socketChannel.socket().getInetAddress().getHostName(), + socketChannel.socket().getPort()); + TestSSLTransportLayer transportLayer = new TestSSLTransportLayer(id, key, sslEngine, netReadBufSize, netWriteBufSize, appBufSize); + transportLayer.startHandshake(); + return transportLayer; + } + + + }; + this.channelBuilder.configure(sslClientConfigs); + this.selector = new Selector(5000, new Metrics(), new MockTime(), "MetricGroup", new LinkedHashMap(), channelBuilder); + } + + private static class CertStores { + + Map sslConfig; + + CertStores(boolean server) throws Exception { + String name = server ? "server" : "client"; + SSLFactory.Mode mode = server ? SSLFactory.Mode.SERVER : SSLFactory.Mode.CLIENT; + File truststoreFile = File.createTempFile(name + "TS", ".jks"); + sslConfig = TestSSLUtils.createSSLConfig(!server, true, mode, truststoreFile, name); + sslConfig.put(SSLConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, Class.forName(SSLConfigs.DEFAULT_PRINCIPAL_BUILDER_CLASS)); + } + + private Map getTrustingConfig(CertStores truststoreConfig) { + Map config = new HashMap(sslConfig); + config.put(SSLConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, truststoreConfig.sslConfig.get(SSLConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG)); + config.put(SSLConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, truststoreConfig.sslConfig.get(SSLConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG)); + config.put(SSLConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, truststoreConfig.sslConfig.get(SSLConfigs.SSL_TRUSTSTORE_TYPE_CONFIG)); + return config; + } + + private Map getUntrustingConfig() { + return sslConfig; + } + } + + /** + * SSLTransportLayer with overrides for packet and application buffer size to test buffer resize + * code path. The overridden buffer size starts with a small value and increases in size when the buffer + * size is retrieved to handle overflow/underflow, until the actual session buffer size is reached. + */ + private static class TestSSLTransportLayer extends SSLTransportLayer { + + private final ResizeableBufferSize netReadBufSize; + private final ResizeableBufferSize netWriteBufSize; + private final ResizeableBufferSize appBufSize; + + public TestSSLTransportLayer(String channelId, SelectionKey key, SSLEngine sslEngine, + Integer netReadBufSize, Integer netWriteBufSize, Integer appBufSize) throws IOException { + super(channelId, key, sslEngine); + this.netReadBufSize = new ResizeableBufferSize(netReadBufSize); + this.netWriteBufSize = new ResizeableBufferSize(netWriteBufSize); + this.appBufSize = new ResizeableBufferSize(appBufSize); + } + + @Override + protected int netReadBufferSize() { + ByteBuffer netReadBuffer = netReadBuffer(); + // netReadBufferSize() is invoked in SSLTransportLayer.read() prior to the read + // operation. To avoid the read buffer being expanded too early, increase buffer size + // only when read buffer is full. This ensures that BUFFER_UNDERFLOW is always + // triggered in testNetReadBufferResize(). + boolean updateBufSize = netReadBuffer != null && !netReadBuffer().hasRemaining(); + return netReadBufSize.updateAndGet(super.netReadBufferSize(), updateBufSize); + } + + @Override + protected int netWriteBufferSize() { + return netWriteBufSize.updateAndGet(super.netWriteBufferSize(), true); + } + + @Override + protected int applicationBufferSize() { + return appBufSize.updateAndGet(super.applicationBufferSize(), true); + } + + private static class ResizeableBufferSize { + private Integer bufSizeOverride; + ResizeableBufferSize(Integer bufSizeOverride) { + this.bufSizeOverride = bufSizeOverride; + } + int updateAndGet(int actualSize, boolean update) { + int size = actualSize; + if (bufSizeOverride != null) { + if (update) + bufSizeOverride = Math.min(bufSizeOverride * 2, size); + size = bufSizeOverride; + } + return size; + } + } + } + + // Non-blocking EchoServer implementation that uses SSLTransportLayer + private class SSLEchoServer extends Thread { + private final int port; + private final ServerSocketChannel serverSocketChannel; + private final List newChannels; + private final List socketChannels; + private final AcceptorThread acceptorThread; + private SSLFactory sslFactory; + private final Selector selector; + private final ConcurrentLinkedQueue inflightSends = new ConcurrentLinkedQueue(); + + public SSLEchoServer(Map configs, String serverHost) throws Exception { + this.sslFactory = new SSLFactory(SSLFactory.Mode.SERVER); + this.sslFactory.configure(configs); + serverSocketChannel = ServerSocketChannel.open(); + serverSocketChannel.configureBlocking(false); + serverSocketChannel.socket().bind(new InetSocketAddress(serverHost, 0)); + this.port = serverSocketChannel.socket().getLocalPort(); + this.socketChannels = Collections.synchronizedList(new ArrayList()); + this.newChannels = Collections.synchronizedList(new ArrayList()); + SSLChannelBuilder channelBuilder = new SSLChannelBuilder(SSLFactory.Mode.SERVER); + channelBuilder.configure(sslServerConfigs); + this.selector = new Selector(5000, new Metrics(), new MockTime(), "MetricGroup", new LinkedHashMap(), channelBuilder); + setName("echoserver"); + setDaemon(true); + acceptorThread = new AcceptorThread(); + } + + @Override + public void run() { + try { + acceptorThread.start(); + while (serverSocketChannel.isOpen()) { + selector.poll(1000); + for (SocketChannel socketChannel : newChannels) { + String id = id(socketChannel); + selector.register(id, socketChannel); + socketChannels.add(socketChannel); + } + newChannels.clear(); + while (true) { + NetworkSend send = inflightSends.peek(); + if (send != null && !selector.channel(send.destination()).hasSend()) { + send = inflightSends.poll(); + selector.send(send); + } else + break; + } + List completedReceives = selector.completedReceives(); + for (NetworkReceive rcv : completedReceives) { + NetworkSend send = new NetworkSend(rcv.source(), rcv.payload()); + if (!selector.channel(send.destination()).hasSend()) + selector.send(send); + else + inflightSends.add(send); + } + } + } catch (IOException e) { + // ignore + } + } + + private String id(SocketChannel channel) { + return channel.socket().getLocalAddress().getHostAddress() + ":" + channel.socket().getLocalPort() + "-" + + channel.socket().getInetAddress().getHostAddress() + ":" + channel.socket().getPort(); + } + + public void closeConnections() throws IOException { + for (SocketChannel channel : socketChannels) + channel.close(); + socketChannels.clear(); + } + + public void close() throws IOException, InterruptedException { + this.serverSocketChannel.close(); + closeConnections(); + acceptorThread.interrupt(); + acceptorThread.join(); + interrupt(); + join(); + } + + private class AcceptorThread extends Thread { + public AcceptorThread() throws IOException { + setName("acceptor"); + } + public void run() { + try { + + java.nio.channels.Selector acceptSelector = java.nio.channels.Selector.open(); + serverSocketChannel.register(acceptSelector, SelectionKey.OP_ACCEPT); + while (serverSocketChannel.isOpen()) { + if (acceptSelector.select(1000) > 0) { + Iterator it = acceptSelector.selectedKeys().iterator(); + while (it.hasNext()) { + SelectionKey key = it.next(); + if (key.isAcceptable()) { + SocketChannel socketChannel = ((ServerSocketChannel) key.channel()).accept(); + socketChannel.configureBlocking(false); + newChannels.add(socketChannel); + selector.wakeup(); + } + } + } + } + } catch (IOException e) { + // ignore + } + } + } + } + +} 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 5c5e3d40819e4..6aa60ce2206a9 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 @@ -3,32 +3,30 @@ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the * License. You may obtain a copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES 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; -import java.io.DataInputStream; -import java.io.DataOutputStream; +import java.util.HashMap; +import java.util.Map; import java.io.IOException; import java.net.InetSocketAddress; import java.net.ServerSocket; -import java.net.Socket; import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; +import java.util.*; import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.config.SSLConfigs; 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; @@ -40,23 +38,32 @@ */ public class SelectorTest { - private static final List EMPTY = new ArrayList(); - private static final int BUFFER_SIZE = 4 * 1024; + protected static final int BUFFER_SIZE = 4 * 1024; - private EchoServer server; - private Selectable selector; + protected EchoServer server; + protected Time time; + protected Selectable selector; + protected ChannelBuilder channelBuilder; + private Metrics metrics; @Before public void setup() throws Exception { - this.server = new EchoServer(); + Map configs = new HashMap(); + configs.put(SSLConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, Class.forName(SSLConfigs.DEFAULT_PRINCIPAL_BUILDER_CLASS)); + this.server = new EchoServer(configs); this.server.start(); - this.selector = new Selector(new Metrics(), new MockTime()); + this.time = new MockTime(); + this.channelBuilder = new PlaintextChannelBuilder(); + this.channelBuilder.configure(configs); + this.metrics = new Metrics(); + this.selector = new Selector(5000, this.metrics, time, "MetricGroup", new LinkedHashMap(), channelBuilder); } @After public void teardown() throws Exception { this.selector.close(); this.server.close(); + this.metrics.close(); } /** @@ -64,7 +71,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); @@ -73,37 +80,23 @@ 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); assertEquals("hello", blockingRequest(node, "hello")); } - /** - * Validate that the client can intentionally disconnect and reconnect - */ - @Test - public void testClientDisconnect() throws Exception { - int node = 0; - blockingConnect(node); - selector.disconnect(node); - selector.poll(10, asList(createSend(node, "hello1"))); - 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)); - blockingConnect(node); - assertEquals("hello2", blockingRequest(node, "hello2")); - } - /** * Sending a request with one already in flight should result in an 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); } /** @@ -111,7 +104,8 @@ 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); } /** @@ -119,7 +113,7 @@ public void testCantSendWithoutConnecting() throws Exception { */ @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); } /** @@ -127,10 +121,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(); } /** @@ -145,20 +142,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); - + connect(Integer.toString(i), addr); // 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()); @@ -166,20 +163,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))); } } } @@ -189,118 +193,133 @@ 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)); } + @Test + public void testLargeMessageSequence() throws Exception { + int bufferSize = 512 * 1024; + String node = "0"; + int reqs = 50; + InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + connect(node, addr); + String requestPrefix = TestUtils.randomString(bufferSize); + sendAndReceive(node, requestPrefix, 0, reqs); + } + + + /** * Test sending an empty string */ @Test public void testEmptyRequest() throws Exception { - int node = 0; + String node = "0"; blockingConnect(node); assertEquals("", blockingRequest(node, "")); } @Test(expected = IllegalStateException.class) public void testExistingConnectionId() throws IOException { - blockingConnect(0); - blockingConnect(0); + 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()); } - private String blockingRequest(int node, String s) throws IOException { - selector.poll(1000L, asList(createSend(node, s))); + + @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); } } + + protected void connect(String node, InetSocketAddress serverAddr) throws IOException { + selector.connect(node, serverAddr, BUFFER_SIZE, BUFFER_SIZE); + } /* connect and wait for the connection to complete */ - private void blockingConnect(int node) throws IOException { - selector.connect(node, new InetSocketAddress("localhost", server.port), BUFFER_SIZE, BUFFER_SIZE); + private void blockingConnect(String node) throws IOException { + blockingConnect(node, new InetSocketAddress("localhost", server.port)); + } + protected void blockingConnect(String node, InetSocketAddress serverAddr) throws IOException { + selector.connect(node, serverAddr, BUFFER_SIZE, BUFFER_SIZE); while (!selector.connected().contains(node)) - selector.poll(10000L, EMPTY); + selector.poll(10000L); + while (!selector.isChannelReady(node)) + selector.poll(10000L); } - private NetworkSend createSend(int node, String s) { + protected NetworkSend createSend(String node, String s) { return new NetworkSend(node, ByteBuffer.wrap(s.getBytes())); } - private String asString(NetworkReceive receive) { + protected String asString(NetworkReceive receive) { return new String(Utils.toArray(receive.payload())); } - /** - * A simple server that takes size delimited byte arrays and just echos them back to the sender. - */ - static class EchoServer extends Thread { - public final int port; - private final ServerSocket serverSocket; - private final List threads; - private final List sockets; - - public EchoServer() throws Exception { - this.port = TestUtils.choosePort(); - this.serverSocket = new ServerSocket(port); - this.threads = Collections.synchronizedList(new ArrayList()); - this.sockets = Collections.synchronizedList(new ArrayList()); - } + private void sendAndReceive(String node, String requestPrefix, int startIndex, int endIndex) throws Exception { + int requests = startIndex; + int responses = startIndex; + selector.send(createSend(node, requestPrefix + "-" + startIndex)); + requests++; + while (responses < endIndex) { + // do the i/o + selector.poll(0L); + assertEquals("No disconnects should have occurred.", 0, selector.disconnected().size()); - public void run() { - try { - while (true) { - final Socket socket = serverSocket.accept(); - sockets.add(socket); - Thread thread = new Thread() { - public void run() { - try { - DataInputStream input = new DataInputStream(socket.getInputStream()); - DataOutputStream output = new DataOutputStream(socket.getOutputStream()); - while (socket.isConnected() && !socket.isClosed()) { - int size = input.readInt(); - byte[] bytes = new byte[size]; - input.readFully(bytes); - output.writeInt(size); - output.write(bytes); - output.flush(); - } - } catch (IOException e) { - // ignore - } finally { - try { - socket.close(); - } catch (IOException e) { - // ignore - } - } - } - }; - thread.start(); - threads.add(thread); - } - } catch (IOException e) { - // ignore + // handle requests and responses of the fast node + for (NetworkReceive receive : selector.completedReceives()) { + assertEquals(requestPrefix + "-" + responses, asString(receive)); + responses++; } - } - - public void closeConnections() throws IOException { - for (Socket socket : sockets) - socket.close(); - } - public void close() throws IOException, InterruptedException { - this.serverSocket.close(); - closeConnections(); - for (Thread t : threads) - t.join(); - join(); + for (int i = 0; i < selector.completedSends().size() && requests < endIndex; i++, requests++) { + selector.send(createSend(node, requestPrefix + "-" + requests)); + } } } + } 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 94a11121e207d..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 @@ -50,7 +50,9 @@ public void testIterator() { 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)) { @@ -71,7 +73,7 @@ public void testIterator() { public static Collection data() { List values = new ArrayList(); for (CompressionType type: CompressionType.values()) - values.add(new Object[] { type }); + 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 2765913d5bfd4..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 @@ -63,7 +63,7 @@ public void testFields() { @Test public void testChecksum() { assertEquals(record.checksum(), record.computeChecksum()); - 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)); @@ -102,7 +102,7 @@ public static Collection data() { 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 index df37fc6d8f0db..b6680137d6425 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -17,72 +17,145 @@ 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.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.SecurityProtocol; import org.junit.Test; import java.lang.reflect.Method; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.HashSet; +import java.util.Set; import static org.junit.Assert.assertEquals; public class RequestResponseTest { @Test - public void testSerialization() throws Exception{ - List requestList = Arrays.asList( + public void testSerialization() throws Exception { + List requestResponseList = Arrays.asList( createRequestHeader(), createResponseHeader(), createConsumerMetadataRequest(), + createConsumerMetadataRequest().getErrorResponse(0, new UnknownServerException()), createConsumerMetadataResponse(), + createControlledShutdownRequest(), + createControlledShutdownResponse(), + createControlledShutdownRequest().getErrorResponse(1, new UnknownServerException()), 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(), - createProduceResponse()); + createProduceRequest().getErrorResponse(0, new UnknownServerException()), + createProduceResponse(), + createStopReplicaRequest(), + createStopReplicaRequest().getErrorResponse(0, new UnknownServerException()), + createStopReplicaResponse(), + createUpdateMetadataRequest(1), + createUpdateMetadataRequest(1).getErrorResponse(1, new UnknownServerException()), + createUpdateMetadataResponse(), + createLeaderAndIsrRequest(), + createLeaderAndIsrRequest().getErrorResponse(0, new UnknownServerException()), + createLeaderAndIsrResponse() + ); - for (AbstractRequestResponse req: requestList) { - ByteBuffer buffer = ByteBuffer.allocate(req.sizeOf()); - req.writeTo(buffer); - buffer.rewind(); + for (AbstractRequestResponse req : requestResponseList) + checkSerialization(req, null); + + checkSerialization(createUpdateMetadataRequest(0), 0); + checkSerialization(createUpdateMetadataRequest(0).getErrorResponse(0, new UnknownServerException()), 0); + } + + private void checkSerialization(AbstractRequestResponse req, Integer version) throws Exception { + ByteBuffer buffer = ByteBuffer.allocate(req.sizeOf()); + req.writeTo(buffer); + buffer.rewind(); + AbstractRequestResponse deserialized; + if (version == null) { 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()); + deserialized = (AbstractRequestResponse) deserializer.invoke(null, buffer); + } else { + Method deserializer = req.getClass().getDeclaredMethod("parse", ByteBuffer.class, Integer.TYPE); + deserialized = (AbstractRequestResponse) deserializer.invoke(null, buffer, version); } + 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()); } + @Test + public void produceResponseVersionTest() { + Map responseData = new HashMap(); + responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse(Errors.NONE.code(), 10000)); + + ProduceResponse v0Response = new ProduceResponse(responseData); + ProduceResponse v1Response = new ProduceResponse(responseData, 10); + assertEquals("Throttle time must be zero", 0, v0Response.getThrottleTime()); + assertEquals("Throttle time must be 10", 10, v1Response.getThrottleTime()); + assertEquals("Should use schema version 0", ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, 0), v0Response.toStruct().schema()); + assertEquals("Should use schema version 1", ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, 1), v1Response.toStruct().schema()); + assertEquals("Response data does not match", responseData, v0Response.responses()); + assertEquals("Response data does not match", responseData, v1Response.responses()); + } + + @Test + public void fetchResponseVersionTest() { + Map responseData = new HashMap(); + responseData.put(new TopicPartition("test", 0), new FetchResponse.PartitionData(Errors.NONE.code(), 1000000, ByteBuffer.allocate(10))); + + FetchResponse v0Response = new FetchResponse(responseData); + FetchResponse v1Response = new FetchResponse(responseData, 10); + assertEquals("Throttle time must be zero", 0, v0Response.getThrottleTime()); + assertEquals("Throttle time must be 10", 10, v1Response.getThrottleTime()); + assertEquals("Should use schema version 0", ProtoUtils.responseSchema(ApiKeys.FETCH.id, 0), v0Response.toStruct().schema()); + assertEquals("Should use schema version 1", ProtoUtils.responseSchema(ApiKeys.FETCH.id, 1), v1Response.toStruct().schema()); + assertEquals("Response data does not match", responseData, v0Response.responseData()); + assertEquals("Response data does not match", responseData, v1Response.responseData()); + } + + private AbstractRequestResponse createRequestHeader() { - return new RequestHeader((short)10, (short)1, "", 10); + return new RequestHeader((short) 10, (short) 1, "", 10); } private AbstractRequestResponse createResponseHeader() { return new ResponseHeader(10); } - private AbstractRequestResponse createConsumerMetadataRequest() { + private AbstractRequest createConsumerMetadataRequest() { return new ConsumerMetadataRequest("test-group"); } private AbstractRequestResponse createConsumerMetadataResponse() { - return new ConsumerMetadataResponse((short)1, new Node(10, "host1", 2014)); + return new ConsumerMetadataResponse((short) 1, new Node(10, "host1", 2014)); } - private AbstractRequestResponse createFetchRequest() { + 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)); @@ -91,27 +164,27 @@ private AbstractRequestResponse createFetchRequest() { private AbstractRequestResponse createFetchResponse() { Map responseData = new HashMap(); - responseData.put(new TopicPartition("test", 0), new FetchResponse.PartitionData((short)0, 1000000, ByteBuffer.allocate(10))); - return new FetchResponse(responseData); + responseData.put(new TopicPartition("test", 0), new FetchResponse.PartitionData(Errors.NONE.code(), 1000000, ByteBuffer.allocate(10))); + return new FetchResponse(responseData, 0); } - private AbstractRequestResponse createHeartBeatRequest() { + private AbstractRequest createHeartBeatRequest() { return new HeartbeatRequest("group1", 1, "consumer1"); } private AbstractRequestResponse createHeartBeatResponse() { - return new HeartbeatResponse((short)0); + return new HeartbeatResponse(Errors.NONE.code()); } - private AbstractRequestResponse createJoinGroupRequest() { + private AbstractRequest createJoinGroupRequest() { return new JoinGroupRequest("group1", 30000, Arrays.asList("topic1"), "consumer1", "strategy1"); } private AbstractRequestResponse createJoinGroupResponse() { - return new JoinGroupResponse((short)0, 1, "consumer1", Arrays.asList(new TopicPartition("test11", 1), new TopicPartition("test2", 1))); + return new JoinGroupResponse(Errors.NONE.code(), 1, "consumer1", Arrays.asList(new TopicPartition("test11", 1), new TopicPartition("test2", 1))); } - private AbstractRequestResponse createListOffsetRequest() { + private AbstractRequest createListOffsetRequest() { Map offsetData = new HashMap(); offsetData.put(new TopicPartition("test", 0), new ListOffsetRequest.PartitionData(1000000L, 10)); return new ListOffsetRequest(-1, offsetData); @@ -119,11 +192,11 @@ private AbstractRequestResponse createListOffsetRequest() { private AbstractRequestResponse createListOffsetResponse() { Map responseData = new HashMap(); - responseData.put(new TopicPartition("test", 0), new ListOffsetResponse.PartitionData((short)0, Arrays.asList(100L))); + responseData.put(new TopicPartition("test", 0), new ListOffsetResponse.PartitionData(Errors.NONE.code(), Arrays.asList(100L))); return new ListOffsetResponse(responseData); } - private AbstractRequestResponse createMetadataRequest() { + private AbstractRequest createMetadataRequest() { return new MetadataRequest(Arrays.asList("topic1")); } @@ -134,40 +207,131 @@ private AbstractRequestResponse createMetadataResponse() { Node[] isr = new Node[1]; isr[0] = node; Cluster cluster = new Cluster(Arrays.asList(node), Arrays.asList(new PartitionInfo("topic1", 1, node, replicas, isr))); - return new MetadataResponse(cluster); + + Map errors = new HashMap(); + errors.put("topic2", Errors.LEADER_NOT_AVAILABLE); + return new MetadataResponse(cluster, errors); } - private AbstractRequestResponse createOffsetCommitRequest() { + private AbstractRequest createOffsetCommitRequest() { Map commitData = new HashMap(); - commitData.put(new TopicPartition("test", 0), new OffsetCommitRequest.PartitionData(100, 1000000, "")); - return new OffsetCommitRequest("group1", 100, "consumer1", commitData); + 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), (short)0); + responseData.put(new TopicPartition("test", 0), Errors.NONE.code()); return new OffsetCommitResponse(responseData); } - private AbstractRequestResponse createOffsetFetchRequest() { + 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, "", (short)0)); + responseData.put(new TopicPartition("test", 0), new OffsetFetchResponse.PartitionData(100L, "", Errors.NONE.code())); return new OffsetFetchResponse(responseData); } - private AbstractRequestResponse createProduceRequest() { + private AbstractRequest createProduceRequest() { Map produceData = new HashMap(); produceData.put(new TopicPartition("test", 0), ByteBuffer.allocate(10)); - return new ProduceRequest((short)0, 5000, produceData); + return new ProduceRequest((short) 1, 5000, produceData); } private AbstractRequestResponse createProduceResponse() { Map responseData = new HashMap(); - responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse((short) 0, 10000)); - return new ProduceResponse(responseData); + responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse(Errors.NONE.code(), 10000)); + return new ProduceResponse(responseData, 0); + } + + private AbstractRequest createStopReplicaRequest() { + Set partitions = new HashSet<>(Arrays.asList(new TopicPartition("test", 0))); + return new StopReplicaRequest(0, 1, true, partitions); + } + + private AbstractRequestResponse createStopReplicaResponse() { + Map responses = new HashMap<>(); + responses.put(new TopicPartition("test", 0), Errors.NONE.code()); + return new StopReplicaResponse(Errors.NONE.code(), responses); + } + + private AbstractRequest createControlledShutdownRequest() { + return new ControlledShutdownRequest(10); } + + private AbstractRequestResponse createControlledShutdownResponse() { + HashSet topicPartitions = new HashSet<>(Arrays.asList( + new TopicPartition("test2", 5), + new TopicPartition("test1", 10) + )); + return new ControlledShutdownResponse(Errors.NONE.code(), topicPartitions); + } + + private AbstractRequest createLeaderAndIsrRequest() { + Map partitionStates = new HashMap<>(); + List isr = Arrays.asList(1, 2); + List replicas = Arrays.asList(1, 2, 3, 4); + partitionStates.put(new TopicPartition("topic5", 105), + new LeaderAndIsrRequest.PartitionState(0, 2, 1, new ArrayList<>(isr), 2, new HashSet<>(replicas))); + partitionStates.put(new TopicPartition("topic5", 1), + new LeaderAndIsrRequest.PartitionState(1, 1, 1, new ArrayList<>(isr), 2, new HashSet<>(replicas))); + partitionStates.put(new TopicPartition("topic20", 1), + new LeaderAndIsrRequest.PartitionState(1, 0, 1, new ArrayList<>(isr), 2, new HashSet<>(replicas))); + + Set leaders = new HashSet<>(Arrays.asList( + new LeaderAndIsrRequest.EndPoint(0, "test0", 1223), + new LeaderAndIsrRequest.EndPoint(1, "test1", 1223) + )); + + return new LeaderAndIsrRequest(1, 10, partitionStates, leaders); + } + + private AbstractRequestResponse createLeaderAndIsrResponse() { + Map responses = new HashMap<>(); + responses.put(new TopicPartition("test", 0), Errors.NONE.code()); + return new LeaderAndIsrResponse(Errors.NONE.code(), responses); + } + + private AbstractRequest createUpdateMetadataRequest(int version) { + Map partitionStates = new HashMap<>(); + List isr = Arrays.asList(1, 2); + List replicas = Arrays.asList(1, 2, 3, 4); + partitionStates.put(new TopicPartition("topic5", 105), + new UpdateMetadataRequest.PartitionState(0, 2, 1, new ArrayList<>(isr), 2, new HashSet<>(replicas))); + partitionStates.put(new TopicPartition("topic5", 1), + new UpdateMetadataRequest.PartitionState(1, 1, 1, new ArrayList<>(isr), 2, new HashSet<>(replicas))); + partitionStates.put(new TopicPartition("topic20", 1), + new UpdateMetadataRequest.PartitionState(1, 0, 1, new ArrayList<>(isr), 2, new HashSet<>(replicas))); + + if (version == 0) { + Set liveBrokers = new HashSet<>(Arrays.asList( + new UpdateMetadataRequest.BrokerEndPoint(0, "host1", 1223), + new UpdateMetadataRequest.BrokerEndPoint(1, "host2", 1234) + )); + + return new UpdateMetadataRequest(1, 10, liveBrokers, partitionStates); + } else { + Map endPoints1 = new HashMap<>(); + endPoints1.put(SecurityProtocol.PLAINTEXT, new UpdateMetadataRequest.EndPoint("host1", 1223)); + + Map endPoints2 = new HashMap<>(); + endPoints2.put(SecurityProtocol.PLAINTEXT, new UpdateMetadataRequest.EndPoint("host1", 1244)); + endPoints2.put(SecurityProtocol.SSL, new UpdateMetadataRequest.EndPoint("host2", 1234)); + + Set liveBrokers = new HashSet<>(Arrays.asList(new UpdateMetadataRequest.Broker(0, endPoints1), + new UpdateMetadataRequest.Broker(1, endPoints2) + )); + + return new UpdateMetadataRequest(1, 10, partitionStates, liveBrokers); + } + } + + private AbstractRequestResponse createUpdateMetadataResponse() { + return new UpdateMetadataResponse(Errors.NONE.code()); + } + + } diff --git a/clients/src/test/java/org/apache/kafka/common/security/auth/KafkaPrincipalTest.java b/clients/src/test/java/org/apache/kafka/common/security/auth/KafkaPrincipalTest.java new file mode 100644 index 0000000000000..051ad04b107f7 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/security/auth/KafkaPrincipalTest.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.common.security.auth; + +import org.junit.Assert; +import org.junit.Test; + +public class KafkaPrincipalTest { + + @Test + public void testPrincipalNameCanContainSeparator() { + String name = "name" + KafkaPrincipal.SEPARATOR + "with" + KafkaPrincipal.SEPARATOR + "in" + KafkaPrincipal.SEPARATOR + "it"; + + KafkaPrincipal principal = KafkaPrincipal.fromString(KafkaPrincipal.USER_TYPE + KafkaPrincipal.SEPARATOR + name); + Assert.assertEquals(KafkaPrincipal.USER_TYPE, principal.getPrincipalType()); + Assert.assertEquals(name, principal.getName()); + } + + @Test + public void testEqualsAndHashCode() { + String name = "KafkaUser"; + KafkaPrincipal principal1 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, name); + KafkaPrincipal principal2 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, name); + + Assert.assertEquals(principal1.hashCode(), principal2.hashCode()); + Assert.assertEquals(principal1, principal2); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/security/ssl/SSLFactoryTest.java b/clients/src/test/java/org/apache/kafka/common/security/ssl/SSLFactoryTest.java new file mode 100644 index 0000000000000..0aec666b20bf0 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/security/ssl/SSLFactoryTest.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.common.security.ssl; + +import javax.net.ssl.*; + +import java.io.File; +import java.util.Map; + +import org.apache.kafka.test.TestSSLUtils; + +import org.junit.Test; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertTrue; + + +/** + * A set of tests for the selector over ssl. These use a test harness that runs a simple socket server that echos back responses. + */ + +public class SSLFactoryTest { + + @Test + public void testSSLFactoryConfiguration() throws Exception { + File trustStoreFile = File.createTempFile("truststore", ".jks"); + Map serverSSLConfig = TestSSLUtils.createSSLConfig(false, true, SSLFactory.Mode.SERVER, trustStoreFile, "server"); + SSLFactory sslFactory = new SSLFactory(SSLFactory.Mode.SERVER); + sslFactory.configure(serverSSLConfig); + //host and port are hints + SSLEngine engine = sslFactory.createSSLEngine("localhost", 0); + assertNotNull(engine); + String[] expectedProtocols = {"TLSv1.2"}; + assertArrayEquals(expectedProtocols, engine.getEnabledProtocols()); + assertEquals(false, engine.getUseClientMode()); + } + + @Test + public void testClientMode() throws Exception { + File trustStoreFile = File.createTempFile("truststore", ".jks"); + Map clientSSLConfig = TestSSLUtils.createSSLConfig(false, true, SSLFactory.Mode.CLIENT, trustStoreFile, "client"); + SSLFactory sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT); + sslFactory.configure(clientSSLConfig); + //host and port are hints + SSLEngine engine = sslFactory.createSSLEngine("localhost", 0); + assertTrue(engine.getUseClientMode()); + } + +} 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/CrcTest.java b/clients/src/test/java/org/apache/kafka/common/utils/CrcTest.java index 6b323819390b5..c39c3cff64d4e 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/CrcTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/CrcTest.java @@ -25,7 +25,7 @@ public class CrcTest { @Test public void testUpdate() { - final byte bytes[] = "Any String you want".getBytes(); + final byte[] bytes = "Any String you want".getBytes(); final int len = bytes.length; Crc32 crc1 = new Crc32(); @@ -33,10 +33,10 @@ public void testUpdate() { Crc32 crc3 = new Crc32(); crc1.update(bytes, 0, len); - for(int i = 0; i < len; i++) + 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); + 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()); 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 old mode 100644 new mode 100755 index a39fab532f731..74ec52bc521d5 --- a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java @@ -16,6 +16,10 @@ */ 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; @@ -28,9 +32,9 @@ public class UtilsTest { @Test public void testGetHost() { assertEquals("127.0.0.1", getHost("127.0.0.1:8000")); - assertEquals("mydomain.com", getHost("mydomain.com:8080")); + assertEquals("mydomain.com", getHost("PLAINTEXT://mydomain.com:8080")); assertEquals("::1", getHost("[::1]:1234")); - assertEquals("2001:db8:85a3:8d3:1319:8a2e:370:7348", getHost("[2001:db8:85a3:8d3:1319:8a2e:370:7348]:5678")); + assertEquals("2001:db8:85a3:8d3:1319:8a2e:370:7348", getHost("PLAINTEXT://[2001:db8:85a3:8d3:1319:8a2e:370:7348]:5678")); } @Test @@ -48,4 +52,60 @@ public void testFormatAddress() { 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)); } -} \ No newline at end of file + + @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)); + } +} 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 9d98c11482554..5222cd08a0c08 100644 --- a/clients/src/test/java/org/apache/kafka/test/MetricsBench.java +++ b/clients/src/test/java/org/apache/kafka/test/MetricsBench.java @@ -14,6 +14,7 @@ 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; @@ -28,23 +29,27 @@ public class MetricsBench { public static void main(String[] args) { long iters = Long.parseLong(args[0]); Metrics metrics = new Metrics(); - 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 Percentiles(1024, - 0.0, - iters, - BucketSizing.CONSTANT, - new Percentile(sensor.name() + ".median", 50.0), - new Percentile(sensor.name() + ".p_99", 99.0))); + try { + Sensor parent = metrics.sensor("parent"); + Sensor child = metrics.sensor("child", parent); + for (Sensor sensor : Arrays.asList(parent, child)) { + 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(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++) + parent.record(i); + double ellapsed = (System.nanoTime() - start) / (double) iters; + System.out.println(String.format("%.2f ns per metric recording.", ellapsed)); + } finally { + metrics.close(); } - long start = System.nanoTime(); - for (int i = 0; i < iters; 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 b24d4de21bfea..8cd19b2edfa75 100644 --- a/clients/src/test/java/org/apache/kafka/test/Microbenchmarks.java +++ b/clients/src/test/java/org/apache/kafka/test/Microbenchmarks.java @@ -162,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 metricRemoval(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..5a5f963ac8849 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 + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this 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.test; @@ -24,40 +20,41 @@ 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) { - this.disconnected.add(id); + public void wakeup() { } @Override - public void wakeup() { + public void close() { } @Override - public void close() { + public void close(String id) { + this.disconnected.add(id); } public void clear() { @@ -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,33 @@ 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() { + } + + @Override + public boolean isChannelReady(String id) { + return true; + } } 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/TestSSLUtils.java b/clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java new file mode 100644 index 0000000000000..c01cf3748ff16 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java @@ -0,0 +1,243 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.config.SSLConfigs; +import org.apache.kafka.common.security.ssl.SSLFactory; +import org.apache.kafka.clients.CommonClientConfigs; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.EOFException; +import java.math.BigInteger; +import javax.net.ssl.TrustManagerFactory; +import java.security.*; +import java.security.cert.Certificate; +import java.security.cert.CertificateException; +import java.security.cert.X509Certificate; + +import org.bouncycastle.asn1.x500.X500Name; +import org.bouncycastle.asn1.x509.AlgorithmIdentifier; +import org.bouncycastle.asn1.x509.SubjectPublicKeyInfo; +import org.bouncycastle.cert.X509CertificateHolder; +import org.bouncycastle.cert.X509v1CertificateBuilder; +import org.bouncycastle.cert.jcajce.JcaX509CertificateConverter; +import org.bouncycastle.crypto.params.AsymmetricKeyParameter; +import org.bouncycastle.crypto.util.PrivateKeyFactory; +import org.bouncycastle.jce.provider.BouncyCastleProvider; +import org.bouncycastle.operator.ContentSigner; +import org.bouncycastle.operator.DefaultDigestAlgorithmIdentifierFinder; +import org.bouncycastle.operator.DefaultSignatureAlgorithmIdentifierFinder; +import org.bouncycastle.operator.bc.BcRSAContentSignerBuilder; + +import java.util.Date; +import java.util.HashMap; +import java.util.Map; +import java.util.List; +import java.util.ArrayList; + + +public class TestSSLUtils { + + /** + * Create a self-signed X.509 Certificate. + * From http://bfo.com/blog/2011/03/08/odds_and_ends_creating_a_new_x_509_certificate.html. + * + * @param dn the X.509 Distinguished Name, eg "CN=Test, L=London, C=GB" + * @param pair the KeyPair + * @param days how many days from now the Certificate is valid for + * @param algorithm the signing algorithm, eg "SHA1withRSA" + * @return the self-signed certificate + * @throws CertificateException thrown if a security error or an IO error ocurred. + */ + public static X509Certificate generateCertificate(String dn, KeyPair pair, + int days, String algorithm) + throws CertificateException { + + try { + Security.addProvider(new BouncyCastleProvider()); + AlgorithmIdentifier sigAlgId = new DefaultSignatureAlgorithmIdentifierFinder().find(algorithm); + AlgorithmIdentifier digAlgId = new DefaultDigestAlgorithmIdentifierFinder().find(sigAlgId); + AsymmetricKeyParameter privateKeyAsymKeyParam = PrivateKeyFactory.createKey(pair.getPrivate().getEncoded()); + SubjectPublicKeyInfo subPubKeyInfo = SubjectPublicKeyInfo.getInstance(pair.getPublic().getEncoded()); + ContentSigner sigGen = new BcRSAContentSignerBuilder(sigAlgId, digAlgId).build(privateKeyAsymKeyParam); + X500Name name = new X500Name(dn); + Date from = new Date(); + Date to = new Date(from.getTime() + days * 86400000L); + BigInteger sn = new BigInteger(64, new SecureRandom()); + + X509v1CertificateBuilder v1CertGen = new X509v1CertificateBuilder(name, sn, from, to, name, subPubKeyInfo); + X509CertificateHolder certificateHolder = v1CertGen.build(sigGen); + return new JcaX509CertificateConverter().setProvider("BC").getCertificate(certificateHolder); + } catch (CertificateException ce) { + throw ce; + } catch (Exception e) { + throw new CertificateException(e); + } + } + + public static KeyPair generateKeyPair(String algorithm) throws NoSuchAlgorithmException { + KeyPairGenerator keyGen = KeyPairGenerator.getInstance(algorithm); + keyGen.initialize(1024); + return keyGen.genKeyPair(); + } + + private static KeyStore createEmptyKeyStore() throws GeneralSecurityException, IOException { + KeyStore ks = KeyStore.getInstance("JKS"); + ks.load(null, null); // initialize + return ks; + } + + private static void saveKeyStore(KeyStore ks, String filename, + String password) throws GeneralSecurityException, IOException { + FileOutputStream out = new FileOutputStream(filename); + try { + ks.store(out, password.toCharArray()); + } finally { + out.close(); + } + } + + public static void createKeyStore(String filename, + String password, String alias, + Key privateKey, Certificate cert) throws GeneralSecurityException, IOException { + KeyStore ks = createEmptyKeyStore(); + ks.setKeyEntry(alias, privateKey, password.toCharArray(), + new Certificate[]{cert}); + saveKeyStore(ks, filename, password); + } + + /** + * Creates a keystore with a single key and saves it to a file. + * + * @param filename String file to save + * @param password String store password to set on keystore + * @param keyPassword String key password to set on key + * @param alias String alias to use for the key + * @param privateKey Key to save in keystore + * @param cert Certificate to use as certificate chain associated to key + * @throws GeneralSecurityException for any error with the security APIs + * @throws IOException if there is an I/O error saving the file + */ + public static void createKeyStore(String filename, + String password, String keyPassword, String alias, + Key privateKey, Certificate cert) throws GeneralSecurityException, IOException { + KeyStore ks = createEmptyKeyStore(); + ks.setKeyEntry(alias, privateKey, keyPassword.toCharArray(), + new Certificate[]{cert}); + saveKeyStore(ks, filename, password); + } + + public static void createTrustStore(String filename, + String password, String alias, + Certificate cert) throws GeneralSecurityException, IOException { + KeyStore ks = createEmptyKeyStore(); + ks.setCertificateEntry(alias, cert); + saveKeyStore(ks, filename, password); + } + + public static void createTrustStore( + String filename, String password, Map certs) throws GeneralSecurityException, IOException { + KeyStore ks = KeyStore.getInstance("JKS"); + try { + FileInputStream in = new FileInputStream(filename); + ks.load(in, password.toCharArray()); + in.close(); + } catch (EOFException e) { + ks = createEmptyKeyStore(); + } + for (Map.Entry cert : certs.entrySet()) { + ks.setCertificateEntry(cert.getKey(), cert.getValue()); + } + saveKeyStore(ks, filename, password); + } + + public static Map createX509Certificates(KeyPair keyPair) + throws GeneralSecurityException { + Map certs = new HashMap(); + X509Certificate cert = generateCertificate("CN=localhost, O=localhost", keyPair, 30, "SHA1withRSA"); + certs.put("localhost", cert); + return certs; + } + + public static Map createSSLConfig(SSLFactory.Mode mode, File keyStoreFile, String password, String keyPassword, + File trustStoreFile, String trustStorePassword) { + Map sslConfigs = new HashMap(); + sslConfigs.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SSL"); // kafka security protocol + sslConfigs.put(SSLConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2"); // protocol to create SSLContext + + if (mode == SSLFactory.Mode.SERVER || (mode == SSLFactory.Mode.CLIENT && keyStoreFile != null)) { + sslConfigs.put(SSLConfigs.SSL_KEYSTORE_LOCATION_CONFIG, keyStoreFile.getPath()); + sslConfigs.put(SSLConfigs.SSL_KEYSTORE_TYPE_CONFIG, "JKS"); + sslConfigs.put(SSLConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, TrustManagerFactory.getDefaultAlgorithm()); + sslConfigs.put(SSLConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, password); + sslConfigs.put(SSLConfigs.SSL_KEY_PASSWORD_CONFIG, keyPassword); + } + + sslConfigs.put(SSLConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, trustStoreFile.getPath()); + sslConfigs.put(SSLConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, trustStorePassword); + sslConfigs.put(SSLConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, "JKS"); + sslConfigs.put(SSLConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, TrustManagerFactory.getDefaultAlgorithm()); + + List enabledProtocols = new ArrayList(); + enabledProtocols.add("TLSv1.2"); + sslConfigs.put(SSLConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, enabledProtocols); + + return sslConfigs; + } + + public static Map createSSLConfig(boolean useClientCert, boolean trustStore, SSLFactory.Mode mode, File trustStoreFile, String certAlias) + throws IOException, GeneralSecurityException { + Map certs = new HashMap(); + File keyStoreFile; + String password; + + if (mode == SSLFactory.Mode.SERVER) + password = "ServerPassword"; + else + password = "ClientPassword"; + + String trustStorePassword = "TrustStorePassword"; + + if (useClientCert) { + keyStoreFile = File.createTempFile("clientKS", ".jks"); + KeyPair cKP = generateKeyPair("RSA"); + X509Certificate cCert = generateCertificate("CN=localhost, O=client", cKP, 30, "SHA1withRSA"); + createKeyStore(keyStoreFile.getPath(), password, "client", cKP.getPrivate(), cCert); + certs.put(certAlias, cCert); + } else { + keyStoreFile = File.createTempFile("serverKS", ".jks"); + KeyPair sKP = generateKeyPair("RSA"); + X509Certificate sCert = generateCertificate("CN=localhost, O=server", sKP, 30, + "SHA1withRSA"); + createKeyStore(keyStoreFile.getPath(), password, password, "server", sKP.getPrivate(), sCert); + certs.put(certAlias, sCert); + } + + if (trustStore) { + createTrustStore(trustStoreFile.getPath(), trustStorePassword, certs); + } + + Map sslConfig = createSSLConfig(mode, keyStoreFile, password, + password, trustStoreFile, trustStorePassword); + return sslConfig; + } + +} 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 76a17e8849bad..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,32 +57,6 @@ 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 * @@ -92,7 +64,7 @@ public static int choosePort() { */ 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/config/copycat-console-sink.properties b/config/copycat-console-sink.properties new file mode 100644 index 0000000000000..4cd4c33a2c004 --- /dev/null +++ b/config/copycat-console-sink.properties @@ -0,0 +1,19 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name=local-console-sink +connector.class=org.apache.kafka.copycat.file.FileStreamSinkConnector +tasks.max=1 +topics=test \ No newline at end of file diff --git a/config/copycat-console-source.properties b/config/copycat-console-source.properties new file mode 100644 index 0000000000000..17dbbf975aa1b --- /dev/null +++ b/config/copycat-console-source.properties @@ -0,0 +1,19 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name=local-console-source +connector.class=org.apache.kafka.copycat.file.FileStreamSourceConnector +tasks.max=1 +topic=test \ No newline at end of file diff --git a/config/copycat-distributed.properties b/config/copycat-distributed.properties new file mode 100644 index 0000000000000..654ed24dad2ef --- /dev/null +++ b/config/copycat-distributed.properties @@ -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. +## + +# These are defaults. This file just demonstrates how to override some settings. +bootstrap.servers=localhost:9092 + +# The converters specify the format of data in Kafka and how to translate it into Copycat data. Every Copycat user will +# need to configure these based on the format they want their data in when loaded from or stored into Kafka +key.converter=org.apache.kafka.copycat.json.JsonConverter +value.converter=org.apache.kafka.copycat.json.JsonConverter +# Converter-specific settings can be passed in by prefixing the Converter's setting with the converter we want to apply +# it to +key.converter.schemas.enable=true +value.converter.schemas.enable=true + +# The offset converter is configurable and must be specified, but most users will always want to use the built-in default. +# Offset data is never visible outside of Copcyat. +offset.key.converter=org.apache.kafka.copycat.json.JsonConverter +offset.value.converter=org.apache.kafka.copycat.json.JsonConverter +offset.key.converter.schemas.enable=false +offset.value.converter.schemas.enable=false + +offset.storage.topic=copycat-offsets +# Flush much faster than normal, which is useful for testing/debugging +offset.flush.interval.ms=10000 diff --git a/config/copycat-file-sink.properties b/config/copycat-file-sink.properties new file mode 100644 index 0000000000000..3cc0d621ba342 --- /dev/null +++ b/config/copycat-file-sink.properties @@ -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. + +name=local-file-sink +connector.class=org.apache.kafka.copycat.file.FileStreamSinkConnector +tasks.max=1 +file=test.sink.txt +topics=test \ No newline at end of file diff --git a/config/copycat-file-source.properties b/config/copycat-file-source.properties new file mode 100644 index 0000000000000..7512e50e0c628 --- /dev/null +++ b/config/copycat-file-source.properties @@ -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. + +name=local-file-source +connector.class=org.apache.kafka.copycat.file.FileStreamSourceConnector +tasks.max=1 +file=test.txt +topic=test \ No newline at end of file diff --git a/config/copycat-log4j.properties b/config/copycat-log4j.properties new file mode 100644 index 0000000000000..158daed315873 --- /dev/null +++ b/config/copycat-log4j.properties @@ -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. + +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:%L)%n + +log4j.logger.org.apache.zookeeper=ERROR +log4j.logger.org.I0Itec.zkclient=ERROR \ No newline at end of file diff --git a/config/copycat-standalone.properties b/config/copycat-standalone.properties new file mode 100644 index 0000000000000..fd264b565f790 --- /dev/null +++ b/config/copycat-standalone.properties @@ -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. + +# These are defaults. This file just demonstrates how to override some settings. +bootstrap.servers=localhost:9092 + +# The converters specify the format of data in Kafka and how to translate it into Copycat data. Every Copycat user will +# need to configure these based on the format they want their data in when loaded from or stored into Kafka +key.converter=org.apache.kafka.copycat.json.JsonConverter +value.converter=org.apache.kafka.copycat.json.JsonConverter +# Converter-specific settings can be passed in by prefixing the Converter's setting with the converter we want to apply +# it to +key.converter.schemas.enable=true +value.converter.schemas.enable=true + +# The offset converter is configurable and must be specified, but most users will always want to use the built-in default. +# Offset data is never visible outside of Copcyat. +offset.key.converter=org.apache.kafka.copycat.json.JsonConverter +offset.value.converter=org.apache.kafka.copycat.json.JsonConverter +offset.key.converter.schemas.enable=false +offset.value.converter.schemas.enable=false + +offset.storage.file.filename=/tmp/copycat.offsets +# Flush much faster than normal, which is useful for testing/debugging +offset.flush.interval.ms=10000 diff --git a/config/log4j.properties b/config/log4j.properties index 95022543debed..bf816e76d2f2f 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 @@ -51,6 +49,12 @@ log4j.appender.controllerAppender.File=${kafka.logs.dir}/controller.log log4j.appender.controllerAppender.layout=org.apache.log4j.PatternLayout log4j.appender.controllerAppender.layout.ConversionPattern=[%d] %p %m (%c)%n +log4j.appender.authorizerAppender=org.apache.log4j.DailyRollingFileAppender +log4j.appender.authorizerAppender.DatePattern='.'yyyy-MM-dd-HH +log4j.appender.authorizerAppender.File=${kafka.logs.dir}/kafka-authorizer.log +log4j.appender.authorizerAppender.layout=org.apache.log4j.PatternLayout +log4j.appender.authorizerAppender.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 @@ -76,3 +80,8 @@ log4j.additivity.kafka.log.LogCleaner=false log4j.logger.state.change.logger=TRACE, stateChangeAppender log4j.additivity.state.change.logger=false + +#Change this to debug to get the actual audit log for authorizer. +log4j.logger.kafka.authorizer.logger=WARN, authorizerAppender +log4j.additivity.kafka.authorizer.logger=false + diff --git a/config/producer.properties b/config/producer.properties index 39d65d7c6c21f..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, lz4, lz4hc. -# the old config values work as well: 0, 1, 2, 3, 4 for none, gzip, snappy, lz4, lz4hc, 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 b0e4496a8ca73..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 @@ -46,7 +48,7 @@ num.io.threads=8 socket.send.buffer.bytes=102400 # The receive buffer (SO_RCVBUF) used by the socket server -socket.receive.buffer.bytes=65536 +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 @@ -118,4 +120,4 @@ log.cleaner.enable=false zookeeper.connect=localhost:2181 # Timeout in ms for connecting to zookeeper -zookeeper.connection.timeout.ms=2000 +zookeeper.connection.timeout.ms=6000 diff --git a/config/tools-log4j.properties b/config/tools-log4j.properties index 52f07c96019b4..b19e343265fc3 100644 --- a/config/tools-log4j.properties +++ b/config/tools-log4j.properties @@ -13,8 +13,9 @@ # See the License for the specific language governing permissions and # limitations under the License. -log4j.rootLogger=WARN, stdout +log4j.rootLogger=WARN, stderr -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.stderr=org.apache.log4j.ConsoleAppender +log4j.appender.stderr.layout=org.apache.log4j.PatternLayout +log4j.appender.stderr.layout.ConversionPattern=[%d] %p %m (%c)%n +log4j.appender.stderr.Target=System.err 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/copycat/api/src/main/java/org/apache/kafka/copycat/connector/Connector.java b/copycat/api/src/main/java/org/apache/kafka/copycat/connector/Connector.java new file mode 100644 index 0000000000000..ae141c4416fe5 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/connector/Connector.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.copycat.connector; + +import org.apache.kafka.common.annotation.InterfaceStability; + +import java.util.List; +import java.util.Properties; + +/** + *

            + * Connectors manage integration of Copycat with another system, either as an input that ingests + * data into Kafka or an output that passes data to an external system. Implementations should + * not use this class directly; they should inherit from SourceConnector or SinkConnector. + *

            + *

            + * Connectors have two primary tasks. First, given some configuration, they are responsible for + * creating configurations for a set of {@link Task}s that split up the data processing. For + * example, a database Connector might create Tasks by dividing the set of tables evenly among + * tasks. Second, they are responsible for monitoring inputs for changes that require + * reconfiguration and notifying the Copycat runtime via the ConnectorContext. Continuing the + * previous example, the connector might periodically check for new tables and notify Copycat of + * additions and deletions. Copycat will then request new configurations and update the running + * Tasks. + *

            + */ +@InterfaceStability.Unstable +public abstract class Connector { + + protected ConnectorContext context; + + /** + * Initialize this connector, using the provided ConnectorContext to notify the runtime of + * input configuration changes. + * @param ctx context object used to interact with the Copycat runtime + */ + public void initialize(ConnectorContext ctx) { + context = ctx; + } + + /** + *

            + * Initialize this connector, using the provided ConnectorContext to notify the runtime of + * input configuration changes and using the provided set of Task configurations. + * This version is only used to recover from failures. + *

            + *

            + * The default implementation ignores the provided Task configurations. During recovery, Copycat will request + * an updated set of configurations and update the running Tasks appropriately. However, Connectors should + * implement special handling of this case if it will avoid unnecessary changes to running Tasks. + *

            + * + * @param ctx context object used to interact with the Copycat runtime + * @param taskConfigs existing task configurations, which may be used when generating new task configs to avoid + * churn in partition to task assignments + */ + public void initialize(ConnectorContext ctx, List taskConfigs) { + context = ctx; + // Ignore taskConfigs. May result in more churn of tasks during recovery if updated configs + // are very different, but reduces the difficulty of implementing a Connector + } + + /** + * Start this Connector. This method will only be called on a clean Connector, i.e. it has + * either just been instantiated and initialized or {@link #stop()} has been invoked. + * + * @param props configuration settings + */ + public abstract void start(Properties props); + + /** + * Reconfigure this Connector. Most implementations will not override this, using the default + * implementation that calls {@link #stop()} followed by {@link #start(Properties)}. + * Implementations only need to override this if they want to handle this process more + * efficiently, e.g. without shutting down network connections to the external system. + * + * @param props new configuration settings + */ + public void reconfigure(Properties props) { + stop(); + start(props); + } + + /** + * Returns the Task implementation for this Connector. + */ + public abstract Class taskClass(); + + /** + * Returns a set of configurations for Tasks based on the current configuration, + * producing at most count configurations. + * + * @param maxTasks maximum number of configurations to generate + * @return configurations for Tasks + */ + public abstract List taskConfigs(int maxTasks); + + /** + * Stop this connector. + */ + public abstract void stop(); +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/connector/ConnectorContext.java b/copycat/api/src/main/java/org/apache/kafka/copycat/connector/ConnectorContext.java new file mode 100644 index 0000000000000..ecba69af8a498 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/connector/ConnectorContext.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.copycat.connector; + +import org.apache.kafka.common.annotation.InterfaceStability; + +/** + * ConnectorContext allows Connectors to proactively interact with the Copycat runtime. + */ +@InterfaceStability.Unstable +public interface ConnectorContext { + /** + * Requests that the runtime reconfigure the Tasks for this source. This should be used to + * indicate to the runtime that something about the input/output has changed (e.g. partitions + * added/removed) and the running Tasks will need to be modified. + */ + void requestTaskReconfiguration(); +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/connector/CopycatRecord.java b/copycat/api/src/main/java/org/apache/kafka/copycat/connector/CopycatRecord.java new file mode 100644 index 0000000000000..0d3e8dc99a9f7 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/connector/CopycatRecord.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.copycat.connector; + +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.copycat.data.Schema; + +/** + *

            + * Base class for records containing data to be copied to/from Kafka. This corresponds closely to + * Kafka's ProducerRecord and ConsumerRecord classes, and holds the data that may be used by both + * sources and sinks (topic, kafkaPartition, key, value). Although both implementations include a + * notion of offset, it is not included here because they differ in type. + *

            + */ +@InterfaceStability.Unstable +public abstract class CopycatRecord { + private final String topic; + private final Integer kafkaPartition; + private final Schema keySchema; + private final Object key; + private final Schema valueSchema; + private final Object value; + + public CopycatRecord(String topic, Integer kafkaPartition, Schema valueSchema, Object value) { + this(topic, kafkaPartition, null, null, valueSchema, value); + } + + public CopycatRecord(String topic, Integer kafkaPartition, Schema keySchema, Object key, Schema valueSchema, Object value) { + this.topic = topic; + this.kafkaPartition = kafkaPartition; + this.keySchema = keySchema; + this.key = key; + this.valueSchema = valueSchema; + this.value = value; + } + + public String topic() { + return topic; + } + + public Integer kafkaPartition() { + return kafkaPartition; + } + + public Object key() { + return key; + } + + public Schema keySchema() { + return keySchema; + } + + public Object value() { + return value; + } + + public Schema valueSchema() { + return valueSchema; + } + + @Override + public String toString() { + return "CopycatRecord{" + + "topic='" + topic + '\'' + + ", kafkaPartition=" + kafkaPartition + + ", key=" + key + + ", value=" + value + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + CopycatRecord that = (CopycatRecord) o; + + if (kafkaPartition != null ? !kafkaPartition.equals(that.kafkaPartition) : that.kafkaPartition != null) + return false; + if (topic != null ? !topic.equals(that.topic) : that.topic != null) + return false; + if (keySchema != null ? !keySchema.equals(that.keySchema) : that.keySchema != null) + return false; + if (key != null ? !key.equals(that.key) : that.key != null) + return false; + if (valueSchema != null ? !valueSchema.equals(that.valueSchema) : that.valueSchema != null) + return false; + 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 + (kafkaPartition != null ? kafkaPartition.hashCode() : 0); + result = 31 * result + (keySchema != null ? keySchema.hashCode() : 0); + result = 31 * result + (key != null ? key.hashCode() : 0); + result = 31 * result + (valueSchema != null ? valueSchema.hashCode() : 0); + result = 31 * result + (value != null ? value.hashCode() : 0); + return result; + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/connector/Task.java b/copycat/api/src/main/java/org/apache/kafka/copycat/connector/Task.java new file mode 100644 index 0000000000000..cdaba0845c224 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/connector/Task.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.copycat.connector; + +import org.apache.kafka.common.annotation.InterfaceStability; + +import java.util.Properties; + +/** + *

            + * Tasks contain the code that actually copies data to/from another system. They receive + * a configuration from their parent Connector, assigning them a fraction of a Copycat job's work. + * The Copycat framework then pushes/pulls data from the Task. The Task must also be able to + * respond to reconfiguration requests. + *

            + *

            + * Task only contains the minimal shared functionality between + * {@link org.apache.kafka.copycat.source.SourceTask} and + * {@link org.apache.kafka.copycat.sink.SinkTask}. + *

            + */ +@InterfaceStability.Unstable +public interface Task { + /** + * Start the Task + * @param props initial configuration + */ + void start(Properties props); + + /** + * Stop this task. + */ + void stop(); +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/data/CopycatSchema.java b/copycat/api/src/main/java/org/apache/kafka/copycat/data/CopycatSchema.java new file mode 100644 index 0000000000000..104abf1c99549 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/data/CopycatSchema.java @@ -0,0 +1,323 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.data; + +import org.apache.kafka.copycat.errors.DataException; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.util.*; + +public class CopycatSchema implements Schema { + /** + * Maps Schema.Types to a list of Java classes that can be used to represent them. + */ + private static final Map> SCHEMA_TYPE_CLASSES = new HashMap<>(); + /** + * Maps known logical types to a list of Java classes that can be used to represent them. + */ + private static final Map> LOGICAL_TYPE_CLASSES = new HashMap<>(); + + /** + * Maps the Java classes to the corresponding Schema.Type. + */ + private static final Map, Type> JAVA_CLASS_SCHEMA_TYPES = new HashMap<>(); + + static { + SCHEMA_TYPE_CLASSES.put(Type.INT8, Arrays.asList((Class) Byte.class)); + SCHEMA_TYPE_CLASSES.put(Type.INT16, Arrays.asList((Class) Short.class)); + SCHEMA_TYPE_CLASSES.put(Type.INT32, Arrays.asList((Class) Integer.class)); + SCHEMA_TYPE_CLASSES.put(Type.INT64, Arrays.asList((Class) Long.class)); + SCHEMA_TYPE_CLASSES.put(Type.FLOAT32, Arrays.asList((Class) Float.class)); + SCHEMA_TYPE_CLASSES.put(Type.FLOAT64, Arrays.asList((Class) Double.class)); + SCHEMA_TYPE_CLASSES.put(Type.BOOLEAN, Arrays.asList((Class) Boolean.class)); + SCHEMA_TYPE_CLASSES.put(Type.STRING, Arrays.asList((Class) String.class)); + // Bytes are special and have 2 representations. byte[] causes problems because it doesn't handle equals() and + // hashCode() like we want objects to, so we support both byte[] and ByteBuffer. Using plain byte[] can cause + // those methods to fail, so ByteBuffers are recommended + SCHEMA_TYPE_CLASSES.put(Type.BYTES, Arrays.asList((Class) byte[].class, (Class) ByteBuffer.class)); + SCHEMA_TYPE_CLASSES.put(Type.ARRAY, Arrays.asList((Class) List.class)); + SCHEMA_TYPE_CLASSES.put(Type.MAP, Arrays.asList((Class) Map.class)); + SCHEMA_TYPE_CLASSES.put(Type.STRUCT, Arrays.asList((Class) Struct.class)); + + for (Map.Entry> schemaClasses : SCHEMA_TYPE_CLASSES.entrySet()) { + for (Class schemaClass : schemaClasses.getValue()) + JAVA_CLASS_SCHEMA_TYPES.put(schemaClass, schemaClasses.getKey()); + } + + LOGICAL_TYPE_CLASSES.put(Decimal.LOGICAL_NAME, Arrays.asList((Class) BigDecimal.class)); + LOGICAL_TYPE_CLASSES.put(Date.LOGICAL_NAME, Arrays.asList((Class) java.util.Date.class)); + LOGICAL_TYPE_CLASSES.put(Time.LOGICAL_NAME, Arrays.asList((Class) java.util.Date.class)); + LOGICAL_TYPE_CLASSES.put(Timestamp.LOGICAL_NAME, Arrays.asList((Class) java.util.Date.class)); + // We don't need to put these into JAVA_CLASS_SCHEMA_TYPES since that's only used to determine schemas for + // schemaless data and logical types will have ambiguous schemas (e.g. many of them use the same Java class) so + // they should not be used without schemas. + } + + // The type of the field + private final Type type; + private final boolean optional; + private final Object defaultValue; + + private final List fields; + private final Map fieldsByName; + + private final Schema keySchema; + private final Schema valueSchema; + + // Optional name and version provide a built-in way to indicate what type of data is included. Most + // useful for structs to indicate the semantics of the struct and map it to some existing underlying + // serializer-specific schema. However, can also be useful in specifying other logical types (e.g. a set is an array + // with additional constraints). + private final String name; + private final Integer version; + // Optional human readable documentation describing this schema. + private final String doc; + private final Map parameters; + + /** + * Construct a Schema. Most users should not construct schemas manually, preferring {@link SchemaBuilder} instead. + */ + public CopycatSchema(Type type, boolean optional, Object defaultValue, String name, Integer version, String doc, Map parameters, List fields, Schema keySchema, Schema valueSchema) { + this.type = type; + this.optional = optional; + this.defaultValue = defaultValue; + this.name = name; + this.version = version; + this.doc = doc; + this.parameters = parameters; + + this.fields = fields; + if (this.fields != null && this.type == Type.STRUCT) { + this.fieldsByName = new HashMap<>(); + for (Field field : fields) + fieldsByName.put(field.name(), field); + } else { + this.fieldsByName = null; + } + + this.keySchema = keySchema; + this.valueSchema = valueSchema; + } + + /** + * Construct a Schema for a primitive type, setting schema parameters, struct fields, and key and value schemas to null. + */ + public CopycatSchema(Type type, boolean optional, Object defaultValue, String name, Integer version, String doc) { + this(type, optional, defaultValue, name, version, doc, null, null, null, null); + } + + /** + * Construct a default schema for a primitive type. The schema is required, has no default value, name, version, + * or documentation. + */ + public CopycatSchema(Type type) { + this(type, false, null, null, null, null); + } + + @Override + public Type type() { + return type; + } + + @Override + public boolean isOptional() { + return optional; + } + + @Override + public Object defaultValue() { + return defaultValue; + } + + @Override + public String name() { + return name; + } + + @Override + public Integer version() { + return version; + } + + @Override + public String doc() { + return doc; + } + + @Override + public Map parameters() { + return parameters; + } + + @Override + public List fields() { + if (type != Type.STRUCT) + throw new DataException("Cannot list fields on non-struct type"); + return fields; + } + + public Field field(String fieldName) { + if (type != Type.STRUCT) + throw new DataException("Cannot look up fields on non-struct type"); + return fieldsByName.get(fieldName); + } + + @Override + public Schema keySchema() { + if (type != Type.MAP) + throw new DataException("Cannot look up key schema on non-map type"); + return keySchema; + } + + @Override + public Schema valueSchema() { + if (type != Type.MAP && type != Type.ARRAY) + throw new DataException("Cannot look up value schema on non-array and non-map type"); + return valueSchema; + } + + + + /** + * Validate that the value can be used with the schema, i.e. that its type matches the schema type and nullability + * requirements. Throws a DataException if the value is invalid. + * @param schema Schema to test + * @param value value to test + */ + public static void validateValue(Schema schema, Object value) { + if (value == null) { + if (!schema.isOptional()) + throw new DataException("Invalid value: null used for required field"); + else + return; + } + + List expectedClasses = LOGICAL_TYPE_CLASSES.get(schema.name()); + + if (expectedClasses == null) + expectedClasses = SCHEMA_TYPE_CLASSES.get(schema.type()); + + if (expectedClasses == null) + throw new DataException("Invalid Java object for schema type " + schema.type() + ": " + value.getClass()); + + boolean foundMatch = false; + for (Class expectedClass : expectedClasses) { + if (expectedClass.isInstance(value)) { + foundMatch = true; + break; + } + } + if (!foundMatch) + throw new DataException("Invalid Java object for schema type " + schema.type() + ": " + value.getClass()); + + switch (schema.type()) { + case STRUCT: + Struct struct = (Struct) value; + if (!struct.schema().equals(schema)) + throw new DataException("Struct schemas do not match."); + struct.validate(); + break; + case ARRAY: + List array = (List) value; + for (Object entry : array) + validateValue(schema.valueSchema(), entry); + break; + case MAP: + Map map = (Map) value; + for (Map.Entry entry : map.entrySet()) { + validateValue(schema.keySchema(), entry.getKey()); + validateValue(schema.valueSchema(), entry.getValue()); + } + break; + } + } + + /** + * Validate that the value can be used for this schema, i.e. that its type matches the schema type and optional + * requirements. Throws a DataException if the value is invalid. + * @param value the value to validate + */ + public void validateValue(Object value) { + validateValue(this, value); + } + + @Override + public CopycatSchema schema() { + return this; + } + + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + CopycatSchema schema = (CopycatSchema) o; + return Objects.equals(optional, schema.optional) && + Objects.equals(type, schema.type) && + Objects.equals(defaultValue, schema.defaultValue) && + Objects.equals(fields, schema.fields) && + Objects.equals(keySchema, schema.keySchema) && + Objects.equals(valueSchema, schema.valueSchema) && + Objects.equals(name, schema.name) && + Objects.equals(version, schema.version) && + Objects.equals(doc, schema.doc) && + Objects.equals(parameters, schema.parameters); + } + + @Override + public int hashCode() { + return Objects.hash(type, optional, defaultValue, fields, keySchema, valueSchema, name, version, doc, parameters); + } + + @Override + public String toString() { + if (name != null) + return "Schema{" + name + ":" + type + "}"; + else + return "Schema{" + type + "}"; + } + + + /** + * Get the {@link Type} associated with the the given class. + * + * @param klass the Class to + * @return the corresponding type, nor null if there is no matching type + */ + public static Type schemaType(Class klass) { + synchronized (JAVA_CLASS_SCHEMA_TYPES) { + Type schemaType = JAVA_CLASS_SCHEMA_TYPES.get(klass); + if (schemaType != null) + return schemaType; + + // Since the lookup only checks the class, we need to also try + for (Map.Entry, Type> entry : JAVA_CLASS_SCHEMA_TYPES.entrySet()) { + try { + klass.asSubclass(entry.getKey()); + // Cache this for subsequent lookups + JAVA_CLASS_SCHEMA_TYPES.put(klass, entry.getValue()); + return entry.getValue(); + } catch (ClassCastException e) { + // Expected, ignore + } + } + } + return null; + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/data/Date.java b/copycat/api/src/main/java/org/apache/kafka/copycat/data/Date.java new file mode 100644 index 0000000000000..4e14659eb7ad0 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/data/Date.java @@ -0,0 +1,76 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.data; + +import org.apache.kafka.copycat.errors.DataException; + +import java.util.Calendar; +import java.util.TimeZone; + +/** + *

            + * A date representing a calendar day with no time of day or timezone. The corresponding Java type is a java.util.Date + * with hours, minutes, seconds, milliseconds set to 0. The underlying representation is an integer representing the + * number of standardized days (based on a number of milliseconds with 24 hours/day, 60 minutes/hour, 60 seconds/minute, + * 1000 milliseconds/second with n) since Unix epoch. + *

            + */ +public class Date { + public static final String LOGICAL_NAME = "org.apache.kafka.copycat.data.Date"; + + private static final long MILLIS_PER_DAY = 24 * 60 * 60 * 1000; + + private static final TimeZone UTC = TimeZone.getTimeZone("UTC"); + + /** + * Returns a SchemaBuilder for a Date. By returning a SchemaBuilder you can override additional schema settings such + * as required/optional, default value, and documentation. + * @return a SchemaBuilder + */ + public static SchemaBuilder builder() { + return SchemaBuilder.int32() + .name(LOGICAL_NAME) + .version(1); + } + + public static final Schema SCHEMA = builder().schema(); + + /** + * Convert a value from its logical format (Date) to it's encoded format. + * @param value the logical value + * @return the encoded value + */ + public static int fromLogical(Schema schema, java.util.Date value) { + if (schema.name() == null || !(schema.name().equals(LOGICAL_NAME))) + throw new DataException("Requested conversion of Date object but the schema does not match."); + Calendar calendar = Calendar.getInstance(UTC); + calendar.setTime(value); + if (calendar.get(Calendar.HOUR_OF_DAY) != 0 || calendar.get(Calendar.MINUTE) != 0 || + calendar.get(Calendar.SECOND) != 0 || calendar.get(Calendar.MILLISECOND) != 0) { + throw new DataException("Copycat Date type should not have any time fields set to non-zero values."); + } + long unixMillis = calendar.getTimeInMillis(); + return (int) (unixMillis / MILLIS_PER_DAY); + } + + public static java.util.Date toLogical(Schema schema, int value) { + if (schema.name() == null || !(schema.name().equals(LOGICAL_NAME))) + throw new DataException("Requested conversion of Date object but the schema does not match."); + return new java.util.Date(value * MILLIS_PER_DAY); + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/data/Decimal.java b/copycat/api/src/main/java/org/apache/kafka/copycat/data/Decimal.java new file mode 100644 index 0000000000000..f23e13eae4a4c --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/data/Decimal.java @@ -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 org.apache.kafka.copycat.data; + +import org.apache.kafka.copycat.errors.DataException; + +import java.math.BigDecimal; +import java.math.BigInteger; + +/** + *

            + * An arbitrary-precision signed decimal number. The value is unscaled * 10 ^ -scale where: + *

              + *
            • unscaled is an integer
            • + *
            • scale is an integer representing how many digits the decimal point should be shifted on the unscaled value
            • + *
            + *

            + *

            + * Decimal does not provide a fixed schema because it is parameterized by the scale, which is fixed on the schema + * rather than being part of the value. + *

            + *

            + * The underlying representation of this type is bytes containing a two's complement integer + *

            + */ +public class Decimal { + public static final String LOGICAL_NAME = "org.apache.kafka.copycat.data.Decimal"; + public static final String SCALE_FIELD = "scale"; + + /** + * Returns a SchemaBuilder for a Decimal with the given scale factor. By returning a SchemaBuilder you can override + * additional schema settings such as required/optional, default value, and documentation. + * @param scale the scale factor to apply to unscaled values + * @return a SchemaBuilder + */ + public static SchemaBuilder builder(int scale) { + return SchemaBuilder.bytes() + .name(LOGICAL_NAME) + .parameter(SCALE_FIELD, ((Integer) scale).toString()) + .version(1); + } + + public static Schema schema(int scale) { + return builder(scale).build(); + } + + /** + * Convert a value from its logical format (BigDecimal) to it's encoded format. + * @param value the logical value + * @return the encoded value + */ + public static byte[] fromLogical(Schema schema, BigDecimal value) { + if (value.scale() != scale(schema)) + throw new DataException("BigDecimal has mismatching scale value for given Decimal schema"); + return value.unscaledValue().toByteArray(); + } + + public static BigDecimal toLogical(Schema schema, byte[] value) { + return new BigDecimal(new BigInteger(value), scale(schema)); + } + + private static int scale(Schema schema) { + String scaleString = schema.parameters().get(SCALE_FIELD); + if (scaleString == null) + throw new DataException("Invalid Decimal schema: scale parameter not found."); + try { + return Integer.parseInt(scaleString); + } catch (NumberFormatException e) { + throw new DataException("Invalid scale parameter found in Decimal schema: ", e); + } + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/data/Field.java b/copycat/api/src/main/java/org/apache/kafka/copycat/data/Field.java new file mode 100644 index 0000000000000..c71cdb4c0672d --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/data/Field.java @@ -0,0 +1,77 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.data; + +import java.util.Objects; + +/** + *

            + * A field in a {@link Struct}, consisting of a field name, index, and {@link Schema} for the field value. + *

            + */ +public class Field { + private final String name; + private final int index; + private final Schema schema; + + public Field(String name, int index, Schema schema) { + this.name = name; + this.index = index; + this.schema = schema; + } + + /** + * Get the name of this field. + * @return the name of this field + */ + public String name() { + return name; + } + + + /** + * Get the index of this field within the struct. + * @return the index of this field + */ + public int index() { + return index; + } + + /** + * Get the schema of this field + * @return the schema of values of this field + */ + public Schema schema() { + return schema; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Field field = (Field) o; + return Objects.equals(index, field.index) && + Objects.equals(name, field.name) && + Objects.equals(schema, field.schema); + } + + @Override + public int hashCode() { + return Objects.hash(name, index, schema); + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/data/Schema.java b/copycat/api/src/main/java/org/apache/kafka/copycat/data/Schema.java new file mode 100644 index 0000000000000..3db01aeb768f0 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/data/Schema.java @@ -0,0 +1,163 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.data; + +import java.util.List; +import java.util.Map; + +/** + *

            + * Definition of an abstract data type. Data types can be primitive types (integer types, floating point types, + * boolean, strings, and bytes) or complex types (typed arrays, maps with one key schema and value schema, + * and structs that have a fixed set of field names each with an associated value schema). Any type can be specified + * as optional, allowing it to be omitted (resulting in null values when it is missing) and can specify a default + * value. + *

            + *

            + * All schemas may have some associated metadata: a name, version, and documentation. These are all considered part + * of the schema itself and included when comparing schemas. Besides adding important metadata, these fields enable + * the specification of logical types that specify additional constraints and semantics (e.g. UNIX timestamps are + * just an int64, but the user needs the know about the additional semantics to interpret it properly). + *

            + *

            + * Schemas can be created directly, but in most cases using {@link SchemaBuilder} will be simpler. + *

            + */ +public interface Schema { + /** + * The type of a schema. These only include the core types; logical types must be determined by checking the schema name. + */ + enum Type { + INT8, INT16, INT32, INT64, FLOAT32, FLOAT64, BOOLEAN, STRING, BYTES, ARRAY, MAP, STRUCT; + + private String name; + + Type() { + this.name = this.name().toLowerCase(); + } + + public String getName() { + return name; + } + + public boolean isPrimitive() { + switch (this) { + case INT8: + case INT16: + case INT32: + case INT64: + case FLOAT32: + case FLOAT64: + case BOOLEAN: + case STRING: + case BYTES: + return true; + } + return false; + } + } + + + Schema INT8_SCHEMA = SchemaBuilder.int8().build(); + Schema INT16_SCHEMA = SchemaBuilder.int16().build(); + Schema INT32_SCHEMA = SchemaBuilder.int32().build(); + Schema INT64_SCHEMA = SchemaBuilder.int64().build(); + Schema FLOAT32_SCHEMA = SchemaBuilder.float32().build(); + Schema FLOAT64_SCHEMA = SchemaBuilder.float64().build(); + Schema BOOLEAN_SCHEMA = SchemaBuilder.bool().build(); + Schema STRING_SCHEMA = SchemaBuilder.string().build(); + Schema BYTES_SCHEMA = SchemaBuilder.bytes().build(); + + Schema OPTIONAL_INT8_SCHEMA = SchemaBuilder.int8().optional().build(); + Schema OPTIONAL_INT16_SCHEMA = SchemaBuilder.int16().optional().build(); + Schema OPTIONAL_INT32_SCHEMA = SchemaBuilder.int32().optional().build(); + Schema OPTIONAL_INT64_SCHEMA = SchemaBuilder.int64().optional().build(); + Schema OPTIONAL_FLOAT32_SCHEMA = SchemaBuilder.float32().optional().build(); + Schema OPTIONAL_FLOAT64_SCHEMA = SchemaBuilder.float64().optional().build(); + Schema OPTIONAL_BOOLEAN_SCHEMA = SchemaBuilder.bool().optional().build(); + Schema OPTIONAL_STRING_SCHEMA = SchemaBuilder.string().optional().build(); + Schema OPTIONAL_BYTES_SCHEMA = SchemaBuilder.bytes().optional().build(); + + /** + * @return the type of this schema + */ + Type type(); + + /** + * @return true if this field is optional, false otherwise + */ + boolean isOptional(); + + /** + * @return the default value for this schema + */ + Object defaultValue(); + + /** + * @return the name of this schema + */ + String name(); + + /** + * Get the optional version of the schema. If a version is included, newer versions *must* be larger than older ones. + * @return the version of this schema + */ + Integer version(); + + /** + * @return the documentation for this schema + */ + String doc(); + + /** + * Get a map of schema parameters. + * @return Map containing parameters for this schema, or null if there are no parameters + */ + Map parameters(); + + /** + * Get the key schema for this map schema. Throws a DataException if this schema is not a map. + * @return the key schema + */ + Schema keySchema(); + + /** + * Get the value schema for this map or array schema. Throws a DataException if this schema is not a map or array. + * @return the value schema + */ + Schema valueSchema(); + + /** + * Get the list of fields for this Schema. Throws a DataException if this schema is not a struct. + * @return the list of fields for this Schema + */ + List fields(); + + /** + * Get a field for this Schema by name. Throws a DataException if this schema is not a struct. + * @param fieldName the name of the field to look up + * @return the Field object for the specified field, or null if there is no field with the given name + */ + Field field(String fieldName); + + /** + * Return a concrete instance of the {@link Schema} + * @return the {@link Schema} + */ + Schema schema(); +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/data/SchemaAndValue.java b/copycat/api/src/main/java/org/apache/kafka/copycat/data/SchemaAndValue.java new file mode 100644 index 0000000000000..368a8cf5afb85 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/data/SchemaAndValue.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.copycat.data; + +import java.util.Objects; + +public class SchemaAndValue { + private final Schema schema; + private final Object value; + + public static final SchemaAndValue NULL = new SchemaAndValue(null, null); + + public SchemaAndValue(Schema schema, Object value) { + this.value = value; + this.schema = schema; + } + + public Schema schema() { + return schema; + } + + public Object value() { + return value; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + SchemaAndValue that = (SchemaAndValue) o; + return Objects.equals(schema, that.schema) && + Objects.equals(value, that.value); + } + + @Override + public int hashCode() { + return Objects.hash(schema, value); + } + + @Override + public String toString() { + return "SchemaAndValue{" + + "schema=" + schema + + ", value=" + value + + '}'; + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/data/SchemaBuilder.java b/copycat/api/src/main/java/org/apache/kafka/copycat/data/SchemaBuilder.java new file mode 100644 index 0000000000000..21ae54c9ce84b --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/data/SchemaBuilder.java @@ -0,0 +1,412 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.data; + +import org.apache.kafka.copycat.errors.DataException; +import org.apache.kafka.copycat.errors.SchemaBuilderException; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +/** + *

            + * SchemaBuilder provides a fluent API for constructing {@link Schema} objects. It allows you to set each of the + * properties for the schema and each call returns the SchemaBuilder so the calls can be chained. When nested types + * are required, use one of the predefined schemas from {@link Schema} or use a second SchemaBuilder inline. + *

            + *

            + * Here is an example of building a struct schema: + *

            + *     Schema dateSchema = SchemaBuilder.struct()
            + *         .name("com.example.CalendarDate").version(2).doc("A calendar date including month, day, and year.")
            + *         .field("month", Schema.STRING_SCHEMA)
            + *         .field("day", Schema.INT8_SCHEMA)
            + *         .field("year", Schema.INT16_SCHEMA)
            + *         .build();
            + *     
            + *

            + *

            + * Here is an example of using a second SchemaBuilder to construct complex, nested types: + *

            + *     Schema userListSchema = SchemaBuilder.array(
            + *         SchemaBuilder.struct().name("com.example.User").field("username", Schema.STRING_SCHEMA).field("id", Schema.INT64_SCHEMA).build()
            + *     ).build();
            + *     
            + *

            + */ +public class SchemaBuilder implements Schema { + private static final String TYPE_FIELD = "type"; + private static final String OPTIONAL_FIELD = "optional"; + private static final String DEFAULT_FIELD = "default"; + private static final String NAME_FIELD = "name"; + private static final String VERSION_FIELD = "version"; + private static final String DOC_FIELD = "doc"; + + + private final Type type; + private Boolean optional = null; + private Object defaultValue = null; + + private List fields = null; + private Schema keySchema = null; + private Schema valueSchema = null; + + private String name; + private Integer version; + // Optional human readable documentation describing this schema. + private String doc; + // Additional parameters for logical types. + private Map parameters; + + private SchemaBuilder(Type type) { + this.type = type; + } + + // Common/metadata fields + + @Override + public boolean isOptional() { + return optional == null ? false : optional; + } + + /** + * Set this schema as optional. + * @return the SchemaBuilder + */ + public SchemaBuilder optional() { + checkNull(OPTIONAL_FIELD, optional); + optional = true; + return this; + } + + /** + * Set this schema as required. This is the default, but this method can be used to make this choice explicit. + * @return the SchemaBuilder + */ + public SchemaBuilder required() { + checkNull(OPTIONAL_FIELD, optional); + optional = false; + return this; + } + + @Override + public Object defaultValue() { + return defaultValue; + } + + /** + * Set the default value for this schema. The value is validated against the schema type, throwing a + * {@link SchemaBuilderException} if it does not match. + * @param value the default value + * @return the SchemaBuilder + */ + public SchemaBuilder defaultValue(Object value) { + checkNull(DEFAULT_FIELD, defaultValue); + checkNotNull(TYPE_FIELD, type, DEFAULT_FIELD); + try { + CopycatSchema.validateValue(this, value); + } catch (DataException e) { + throw new SchemaBuilderException("Invalid default value", e); + } + defaultValue = value; + return this; + } + + @Override + public String name() { + return name; + } + + /** + * Set the name of this schema. + * @param name the schema name + * @return the SchemaBuilder + */ + public SchemaBuilder name(String name) { + checkNull(NAME_FIELD, this.name); + this.name = name; + return this; + } + + @Override + public Integer version() { + return version; + } + + /** + * Set the version of this schema. Schema versions are integers which, if provided, must indicate which schema is + * newer and which is older by their ordering. + * @param version the schema version + * @return the SchemaBuilder + */ + public SchemaBuilder version(Integer version) { + checkNull(VERSION_FIELD, this.version); + this.version = version; + return this; + } + + @Override + public String doc() { + return doc; + } + + /** + * Set the documentation for this schema. + * @param doc the documentation + * @return the SchemaBuilder + */ + public SchemaBuilder doc(String doc) { + checkNull(DOC_FIELD, this.doc); + this.doc = doc; + return this; + } + + @Override + public Map parameters() { + return Collections.unmodifiableMap(parameters); + } + + /** + * Set a schema parameter. + * @param propertyName name of the schema property to define + * @param propertyValue value of the schema property to define, as a String + * @return the SchemaBuilder + */ + public SchemaBuilder parameter(String propertyName, String propertyValue) { + // Preserve order of insertion with a LinkedHashMap. This isn't strictly necessary, but is nice if logical types + // can print their properties in a consistent order. + if (parameters == null) + parameters = new LinkedHashMap<>(); + parameters.put(propertyName, propertyValue); + return this; + } + + /** + * Set schema parameters. This operation is additive; it does not remove existing parameters that do not appear in + * the set of properties pass to this method. + * @param props Map of properties to set + * @return the SchemaBuilder + */ + public SchemaBuilder parameters(Map props) { + // Avoid creating an empty set of properties so we never have an empty map + if (props.isEmpty()) + return this; + if (parameters == null) + parameters = new LinkedHashMap<>(); + parameters.putAll(props); + return this; + } + + @Override + public Type type() { + return type; + } + + /** + * Create a SchemaBuilder for the specified type. + * + * Usually it will be simpler to use one of the variants like {@link #string()} or {@link #struct()}, but this form + * can be useful when generating schemas dynamically. + * + * @param type the schema type + * @return a new SchemaBuilder + */ + public static SchemaBuilder type(Type type) { + return new SchemaBuilder(type); + } + + // Primitive types + + /** + * @return a new {@link Type#INT8} SchemaBuilder + */ + public static SchemaBuilder int8() { + return new SchemaBuilder(Type.INT8); + } + + /** + * @return a new {@link Type#INT16} SchemaBuilder + */ + public static SchemaBuilder int16() { + return new SchemaBuilder(Type.INT16); + } + + /** + * @return a new {@link Type#INT32} SchemaBuilder + */ + public static SchemaBuilder int32() { + return new SchemaBuilder(Type.INT32); + } + + /** + * @return a new {@link Type#INT64} SchemaBuilder + */ + public static SchemaBuilder int64() { + return new SchemaBuilder(Type.INT64); + } + + /** + * @return a new {@link Type#FLOAT32} SchemaBuilder + */ + public static SchemaBuilder float32() { + return new SchemaBuilder(Type.FLOAT32); + } + + /** + * @return a new {@link Type#FLOAT64} SchemaBuilder + */ + public static SchemaBuilder float64() { + return new SchemaBuilder(Type.FLOAT64); + } + + /** + * @return a new {@link Type#BOOLEAN} SchemaBuilder + */ + public static SchemaBuilder bool() { + return new SchemaBuilder(Type.BOOLEAN); + } + + /** + * @return a new {@link Type#STRING} SchemaBuilder + */ + public static SchemaBuilder string() { + return new SchemaBuilder(Type.STRING); + } + + /** + * @return a new {@link Type#BYTES} SchemaBuilder + */ + public static SchemaBuilder bytes() { + return new SchemaBuilder(Type.BYTES); + } + + + // Structs + + /** + * @return a new {@link Type#STRUCT} SchemaBuilder + */ + public static SchemaBuilder struct() { + return new SchemaBuilder(Type.STRUCT); + } + + /** + * Add a field to this struct schema. Throws a SchemaBuilderException if this is not a struct schema. + * @param fieldName the name of the field to add + * @param fieldSchema the Schema for the field's value + * @return the SchemaBuilder + */ + public SchemaBuilder field(String fieldName, Schema fieldSchema) { + if (type != Type.STRUCT) + throw new SchemaBuilderException("Cannot create fields on type " + type); + if (fields == null) + fields = new ArrayList<>(); + int fieldIndex = fields.size(); + fields.add(new Field(fieldName, fieldIndex, fieldSchema)); + return this; + } + + /** + * Get the list of fields for this Schema. Throws a DataException if this schema is not a struct. + * @return the list of fields for this Schema + */ + public List fields() { + if (type != Type.STRUCT) + throw new DataException("Cannot list fields on non-struct type"); + return fields; + } + + public Field field(String fieldName) { + if (type != Type.STRUCT) + throw new DataException("Cannot look up fields on non-struct type"); + for (Field field : fields) + if (field.name() == fieldName) + return field; + return null; + } + + + + // Maps & Arrays + + /** + * @param valueSchema the schema for elements of the array + * @return a new {@link Type#ARRAY} SchemaBuilder + */ + public static SchemaBuilder array(Schema valueSchema) { + SchemaBuilder builder = new SchemaBuilder(Type.ARRAY); + builder.valueSchema = valueSchema; + return builder; + } + + /** + * @param keySchema the schema for keys in the map + * @param valueSchema the schema for values in the map + * @return a new {@link Type#MAP} SchemaBuilder + */ + public static SchemaBuilder map(Schema keySchema, Schema valueSchema) { + SchemaBuilder builder = new SchemaBuilder(Type.MAP); + builder.keySchema = keySchema; + builder.valueSchema = valueSchema; + return builder; + } + + @Override + public Schema keySchema() { + return keySchema; + } + + @Override + public Schema valueSchema() { + return valueSchema; + } + + + /** + * Build the Schema using the current settings + * @return the {@link Schema} + */ + public Schema build() { + return new CopycatSchema(type, isOptional(), defaultValue, name, version, doc, + parameters == null ? null : Collections.unmodifiableMap(parameters), + fields == null ? null : Collections.unmodifiableList(fields), keySchema, valueSchema); + } + + /** + * Return a concrete instance of the {@link Schema} specified by this builder + * @return the {@link Schema} + */ + @Override + public Schema schema() { + return build(); + } + + + private static void checkNull(String fieldName, Object val) { + if (val != null) + throw new SchemaBuilderException("Invalid SchemaBuilder call: " + fieldName + " has already been set."); + } + + private static void checkNotNull(String fieldName, Object val, String fieldToSet) { + if (val == null) + throw new SchemaBuilderException("Invalid SchemaBuilder call: " + fieldName + " must be specified to set " + fieldToSet); + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/data/Struct.java b/copycat/api/src/main/java/org/apache/kafka/copycat/data/Struct.java new file mode 100644 index 0000000000000..bd757c4a4d2bd --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/data/Struct.java @@ -0,0 +1,265 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.data; + +import org.apache.kafka.copycat.errors.DataException; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + *

            + * A structured record containing a set of named fields with values, each field using an independent {@link Schema}. + * Struct objects must specify a complete {@link Schema} up front, and only fields specified in the Schema may be set. + *

            + *

            + * The Struct's {@link #put(String, Object)} method returns the Struct itself to provide a fluent API for constructing + * complete objects: + *

            + *         Schema schema = SchemaBuilder.struct().name("com.example.Person")
            + *             .field("name", Schema.STRING_SCHEMA).field("age", Schema.INT32_SCHEMA).build()
            + *         Struct struct = new Struct(schema).put("name", "Bobby McGee").put("age", 21)
            + *     
            + *

            + */ +public class Struct { + + private final Schema schema; + private final Object[] values; + + /** + * Create a new Struct for this {@link Schema} + * @param schema the {@link Schema} for the Struct + */ + public Struct(Schema schema) { + if (schema.type() != Schema.Type.STRUCT) + throw new DataException("Not a struct schema: " + schema); + this.schema = schema; + this.values = new Object[schema.fields().size()]; + } + + /** + * Get the schema for this Struct. + * @return the Struct's schema + */ + public Schema schema() { + return schema; + } + + /** + * Get the value of a field, returning the default value if no value has been set yet and a default value is specified + * in the field's schema. Because this handles fields of all types, the value is returned as an {@link Object} and + * must be cast to a more specific type. + * @param fieldName the field name to lookup + * @return the value for the field + */ + public Object get(String fieldName) { + Field field = lookupField(fieldName); + return get(field); + } + + /** + * Get the value of a field, returning the default value if no value has been set yet and a default value is specified + * in the field's schema. Because this handles fields of all types, the value is returned as an {@link Object} and + * must be cast to a more specific type. + * @param field the field to lookup + * @return the value for the field + */ + public Object get(Field field) { + Object val = values[field.index()]; + if (val == null && schema.defaultValue() != null) { + val = schema.defaultValue(); + } + return val; + } + + /** + * Get the underlying raw value for the field without accounting for default values. + * @param fieldName the field to get the value of + * @return the raw value + */ + public Object getWithoutDefault(String fieldName) { + Field field = lookupField(fieldName); + return values[field.index()]; + } + + // Note that all getters have to have boxed return types since the fields might be optional + + /** + * Equivalent to calling {@link #get(String)} and casting the result to a Byte. + */ + public Byte getInt8(String fieldName) { + return (Byte) getCheckType(fieldName, Schema.Type.INT8); + } + + /** + * Equivalent to calling {@link #get(String)} and casting the result to a Short. + */ + public Short getInt16(String fieldName) { + return (Short) getCheckType(fieldName, Schema.Type.INT16); + } + + /** + * Equivalent to calling {@link #get(String)} and casting the result to a Integer. + */ + public Integer getInt32(String fieldName) { + return (Integer) getCheckType(fieldName, Schema.Type.INT32); + } + + /** + * Equivalent to calling {@link #get(String)} and casting the result to a Long. + */ + public Long getInt64(String fieldName) { + return (Long) getCheckType(fieldName, Schema.Type.INT64); + } + + /** + * Equivalent to calling {@link #get(String)} and casting the result to a Float. + */ + public Float getFloat32(String fieldName) { + return (Float) getCheckType(fieldName, Schema.Type.FLOAT32); + } + + /** + * Equivalent to calling {@link #get(String)} and casting the result to a Double. + */ + public Double getFloat64(String fieldName) { + return (Double) getCheckType(fieldName, Schema.Type.FLOAT64); + } + + /** + * Equivalent to calling {@link #get(String)} and casting the result to a Boolean. + */ + public Boolean getBoolean(String fieldName) { + return (Boolean) getCheckType(fieldName, Schema.Type.BOOLEAN); + } + + /** + * Equivalent to calling {@link #get(String)} and casting the result to a String. + */ + public String getString(String fieldName) { + return (String) getCheckType(fieldName, Schema.Type.STRING); + } + + /** + * Equivalent to calling {@link #get(String)} and casting the result to a byte[]. + */ + public byte[] getBytes(String fieldName) { + Object bytes = getCheckType(fieldName, Schema.Type.BYTES); + if (bytes instanceof ByteBuffer) + return ((ByteBuffer) bytes).array(); + return (byte[]) bytes; + } + + /** + * Equivalent to calling {@link #get(String)} and casting the result to a List. + */ + public List getArray(String fieldName) { + return (List) getCheckType(fieldName, Schema.Type.ARRAY); + } + + /** + * Equivalent to calling {@link #get(String)} and casting the result to a Map. + */ + public Map getMap(String fieldName) { + return (Map) getCheckType(fieldName, Schema.Type.MAP); + } + + /** + * Equivalent to calling {@link #get(String)} and casting the result to a Struct. + */ + public Struct getStruct(String fieldName) { + return (Struct) getCheckType(fieldName, Schema.Type.STRUCT); + } + + /** + * Set the value of a field. Validates the value, throwing a {@link DataException} if it does not match the field's + * {@link Schema}. + * @param fieldName the name of the field to set + * @param value the value of the field + * @return the Struct, to allow chaining of {@link #put(String, Object)} calls + */ + public Struct put(String fieldName, Object value) { + Field field = lookupField(fieldName); + return put(field, value); + } + + /** + * Set the value of a field. Validates the value, throwing a {@link DataException} if it does not match the field's + * {@link Schema}. + * @param field the field to set + * @param value the value of the field + * @return the Struct, to allow chaining of {@link #put(String, Object)} calls + */ + public Struct put(Field field, Object value) { + CopycatSchema.validateValue(field.schema(), value); + values[field.index()] = value; + return this; + } + + + /** + * Validates that this struct has filled in all the necessary data with valid values. For required fields + * without defaults, this validates that a value has been set and has matching types/schemas. If any validation + * fails, throws a DataException. + */ + public void validate() { + for (Field field : schema.fields()) { + Schema fieldSchema = field.schema(); + Object value = values[field.index()]; + if (value == null && (fieldSchema.isOptional() || fieldSchema.defaultValue() != null)) + continue; + CopycatSchema.validateValue(fieldSchema, value); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Struct struct = (Struct) o; + return Objects.equals(schema, struct.schema) && + Arrays.equals(values, struct.values); + } + + @Override + public int hashCode() { + return Objects.hash(schema, Arrays.hashCode(values)); + } + + private Field lookupField(String fieldName) { + Field field = schema.field(fieldName); + if (field == null) + throw new DataException(fieldName + " is not a valid field name"); + return field; + } + + // Get the field's value, but also check that the field matches the specified type, throwing an exception if it doesn't. + // Used to implement the get*() methods that return typed data instead of Object + private Object getCheckType(String fieldName, Schema.Type type) { + Field field = lookupField(fieldName); + if (field.schema().type() != type) + throw new DataException("Field '" + fieldName + "' is not of type " + type); + return values[field.index()]; + } + +} + diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/data/Time.java b/copycat/api/src/main/java/org/apache/kafka/copycat/data/Time.java new file mode 100644 index 0000000000000..e3255e03960f7 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/data/Time.java @@ -0,0 +1,77 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.data; + +import org.apache.kafka.copycat.errors.DataException; + +import java.util.Calendar; +import java.util.TimeZone; + +/** + *

            + * A time representing a specific point in a day, not tied to any specific date. The corresponding Java type is a + * java.util.Date where only hours, minutes, seconds, and milliseconds can be non-zero. This effectively makes it a + * point in time during the first day after the Unix epoch. The underlying representation is an integer + * representing the number of milliseconds after midnight. + *

            + */ +public class Time { + public static final String LOGICAL_NAME = "org.apache.kafka.copycat.data.Time"; + + private static final long MILLIS_PER_DAY = 24 * 60 * 60 * 1000; + + private static final TimeZone UTC = TimeZone.getTimeZone("UTC"); + + /** + * Returns a SchemaBuilder for a Time. By returning a SchemaBuilder you can override additional schema settings such + * as required/optional, default value, and documentation. + * @return a SchemaBuilder + */ + public static SchemaBuilder builder() { + return SchemaBuilder.int32() + .name(LOGICAL_NAME) + .version(1); + } + + public static final Schema SCHEMA = builder().schema(); + + /** + * Convert a value from its logical format (Time) to it's encoded format. + * @param value the logical value + * @return the encoded value + */ + public static int fromLogical(Schema schema, java.util.Date value) { + if (schema.name() == null || !(schema.name().equals(LOGICAL_NAME))) + throw new DataException("Requested conversion of Time object but the schema does not match."); + Calendar calendar = Calendar.getInstance(UTC); + calendar.setTime(value); + long unixMillis = calendar.getTimeInMillis(); + if (unixMillis < 0 || unixMillis > MILLIS_PER_DAY) { + throw new DataException("Copycat Time type should not have any date fields set to non-zero values."); + } + return (int) unixMillis; + } + + public static java.util.Date toLogical(Schema schema, int value) { + if (schema.name() == null || !(schema.name().equals(LOGICAL_NAME))) + throw new DataException("Requested conversion of Date object but the schema does not match."); + if (value < 0 || value > MILLIS_PER_DAY) + throw new DataException("Time values must use number of milliseconds greater than 0 and less than 86400000"); + return new java.util.Date(value); + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/data/Timestamp.java b/copycat/api/src/main/java/org/apache/kafka/copycat/data/Timestamp.java new file mode 100644 index 0000000000000..62d371c2acb89 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/data/Timestamp.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.copycat.data; + +import org.apache.kafka.copycat.errors.DataException; + +import java.util.TimeZone; + +/** + *

            + * A timestamp representing an absolute time, without timezone information. The corresponding Java type is a + * java.util.Date. The underlying representation is a long representing the number of milliseconds since Unix epoch. + *

            + */ +public class Timestamp { + public static final String LOGICAL_NAME = "org.apache.kafka.copycat.data.Timestamp"; + + private static final TimeZone UTC = TimeZone.getTimeZone("UTC"); + + /** + * Returns a SchemaBuilder for a Timestamp. By returning a SchemaBuilder you can override additional schema settings such + * as required/optional, default value, and documentation. + * @return a SchemaBuilder + */ + public static SchemaBuilder builder() { + return SchemaBuilder.int64() + .name(LOGICAL_NAME) + .version(1); + } + + public static final Schema SCHEMA = builder().schema(); + + /** + * Convert a value from its logical format (Date) to it's encoded format. + * @param value the logical value + * @return the encoded value + */ + public static long fromLogical(Schema schema, java.util.Date value) { + if (schema.name() == null || !(schema.name().equals(LOGICAL_NAME))) + throw new DataException("Requested conversion of Timestamp object but the schema does not match."); + return value.getTime(); + } + + public static java.util.Date toLogical(Schema schema, long value) { + if (schema.name() == null || !(schema.name().equals(LOGICAL_NAME))) + throw new DataException("Requested conversion of Timestamp object but the schema does not match."); + return new java.util.Date(value); + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/errors/CopycatException.java b/copycat/api/src/main/java/org/apache/kafka/copycat/errors/CopycatException.java new file mode 100644 index 0000000000000..c8f1bad98cec4 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/errors/CopycatException.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.copycat.errors; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.annotation.InterfaceStability; + +/** + * CopycatException is the top-level exception type generated by Copycat and connectors. + */ +@InterfaceStability.Unstable +public class CopycatException extends KafkaException { + + public CopycatException(String s) { + super(s); + } + + public CopycatException(String s, Throwable throwable) { + super(s, throwable); + } + + public CopycatException(Throwable throwable) { + super(throwable); + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/errors/DataException.java b/copycat/api/src/main/java/org/apache/kafka/copycat/errors/DataException.java new file mode 100644 index 0000000000000..11139a4a04e9f --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/errors/DataException.java @@ -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 org.apache.kafka.copycat.errors; + +/** + * Base class for all Copycat data API exceptions. + */ +public class DataException extends CopycatException { + public DataException(String s) { + super(s); + } + + public DataException(String s, Throwable throwable) { + super(s, throwable); + } + + public DataException(Throwable throwable) { + super(throwable); + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/errors/IllegalWorkerStateException.java b/copycat/api/src/main/java/org/apache/kafka/copycat/errors/IllegalWorkerStateException.java new file mode 100644 index 0000000000000..6f9f23309792c --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/errors/IllegalWorkerStateException.java @@ -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 org.apache.kafka.copycat.errors; + +/** + * Indicates that a method has been invoked illegally or at an invalid time by a connector or task. + */ +public class IllegalWorkerStateException extends CopycatException { + public IllegalWorkerStateException(String s) { + super(s); + } + + public IllegalWorkerStateException(String s, Throwable throwable) { + super(s, throwable); + } + + public IllegalWorkerStateException(Throwable throwable) { + super(throwable); + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/errors/SchemaBuilderException.java b/copycat/api/src/main/java/org/apache/kafka/copycat/errors/SchemaBuilderException.java new file mode 100644 index 0000000000000..b5a93af51551d --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/errors/SchemaBuilderException.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.copycat.errors; + +public class SchemaBuilderException extends DataException { + public SchemaBuilderException(String s) { + super(s); + } + + public SchemaBuilderException(String s, Throwable throwable) { + super(s, throwable); + } + + public SchemaBuilderException(Throwable throwable) { + super(throwable); + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkConnector.java b/copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkConnector.java new file mode 100644 index 0000000000000..fb2e694df99e1 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkConnector.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.copycat.sink; + +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.copycat.connector.Connector; + +/** + * SinkConnectors implement the Connector interface to send Kafka data to another system. + */ +@InterfaceStability.Unstable +public abstract class SinkConnector extends Connector { + + /** + *

            + * Configuration key for the list of input topics for this connector. + *

            + *

            + * Usually this setting is only relevant to the Copycat framework, but is provided here for + * the convenience of Connector developers if they also need to know the set of topics. + *

            + */ + public static final String TOPICS_CONFIG = "topics"; + +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkRecord.java b/copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkRecord.java new file mode 100644 index 0000000000000..79ac725577590 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkRecord.java @@ -0,0 +1,72 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

            + * http://www.apache.org/licenses/LICENSE-2.0 + *

            + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.sink; + +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.copycat.connector.CopycatRecord; +import org.apache.kafka.copycat.data.Schema; + +/** + * SinkRecord is a CopycatRecord that has been read from Kafka and includes the kafkaOffset of + * the record in the Kafka topic-partition in addition to the standard fields. This information + * should be used by the SinkTask to coordinate kafkaOffset commits. + */ +@InterfaceStability.Unstable +public class SinkRecord extends CopycatRecord { + private final long kafkaOffset; + + public SinkRecord(String topic, int partition, Schema keySchema, Object key, Schema valueSchema, Object value, long kafkaOffset) { + super(topic, partition, keySchema, key, valueSchema, value); + this.kafkaOffset = kafkaOffset; + } + + public long kafkaOffset() { + return kafkaOffset; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + if (!super.equals(o)) + return false; + + SinkRecord that = (SinkRecord) o; + + if (kafkaOffset != that.kafkaOffset) + return false; + + return true; + } + + @Override + public int hashCode() { + int result = super.hashCode(); + result = 31 * result + (int) (kafkaOffset ^ (kafkaOffset >>> 32)); + return result; + } + + @Override + public String toString() { + return "SinkRecord{" + + "kafkaOffset=" + kafkaOffset + + "} " + super.toString(); + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkTask.java b/copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkTask.java new file mode 100644 index 0000000000000..bf5d152100d04 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkTask.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.copycat.sink; + +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.copycat.connector.Task; + +import java.util.Collection; +import java.util.Map; + +/** + * SinkTask is a Task takes records loaded from Kafka and sends them to another system. In + * addition to the basic {@link #put} interface, SinkTasks must also implement {@link #flush} + * to support offset commits. + */ +@InterfaceStability.Unstable +public abstract class SinkTask implements Task { + + /** + *

            + * The configuration key that provides the list of topics that are inputs for this + * SinkTask. + *

            + */ + public static final String TOPICS_CONFIG = "topics"; + + protected SinkTaskContext context; + + public void initialize(SinkTaskContext context) { + this.context = context; + } + + /** + * Put the records in the sink. Usually this should send the records to the sink asynchronously + * and immediately return. + * + * @param records the set of records to send + */ + public abstract void put(Collection records); + + /** + * Flush all records that have been {@link #put} for the specified topic-partitions. The + * offsets are provided for convenience, but could also be determined by tracking all offsets + * included in the SinkRecords passed to {@link #put}. + * + * @param offsets mapping of TopicPartition to committed offset + */ + public abstract void flush(Map offsets); +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkTaskContext.java b/copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkTaskContext.java new file mode 100644 index 0000000000000..3ecff2708fadd --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkTaskContext.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.copycat.sink; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.annotation.InterfaceStability; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +/** + * Context passed to SinkTasks, allowing them to access utilities in the copycat runtime. + */ +@InterfaceStability.Unstable +public abstract class SinkTaskContext { + private Map offsets; + + public SinkTaskContext() { + offsets = new HashMap<>(); + } + + /** + * Reset the consumer offsets for the given topic partitions. SinkTasks should use this when they are started + * if they manage offsets in the sink data store rather than using Kafka consumer offsets. For example, an HDFS + * connector might record offsets in HDFS to provide exactly once delivery. When the SinkTask is started or + * a rebalance occurs, the task would reload offsets from HDFS and use this method to reset the consumer to those + * offsets. + * + * SinkTasks that do not manage their own offsets do not need to use this method. + * + * @param offsets map of offsets for topic partitions + */ + public void offset(Map offsets) { + this.offsets = offsets; + } + + /** + * Get offsets that the SinkTask has submitted to be reset. Used by the Copycat framework. + * @return the map of offsets + */ + public Map offsets() { + return offsets; + } + + /** + * Get the current set of assigned TopicPartitions for this task. + * @return the set of currently assigned TopicPartitions + */ + public abstract Set assignment(); + + /** + * Pause consumption of messages from the specified TopicPartitions. + * @param partitions the partitions which should be paused + */ + public abstract void pause(TopicPartition... partitions); + + /** + * Resume consumption of messages from previously paused TopicPartitions. + * @param partitions the partitions to resume + */ + public abstract void resume(TopicPartition... partitions); +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceConnector.java b/copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceConnector.java new file mode 100644 index 0000000000000..7258cdfb826f3 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceConnector.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.copycat.source; + +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.copycat.connector.Connector; + +/** + * SourceConnectors implement the connector interface to pull data from another system and send + * it to Kafka. + */ +@InterfaceStability.Unstable +public abstract class SourceConnector extends Connector { + +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceRecord.java b/copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceRecord.java new file mode 100644 index 0000000000000..7f54c10c7de56 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceRecord.java @@ -0,0 +1,109 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

            + * http://www.apache.org/licenses/LICENSE-2.0 + *

            + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.source; + +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.copycat.connector.CopycatRecord; +import org.apache.kafka.copycat.data.Schema; + +import java.util.Map; + +/** + *

            + * SourceRecords are generated by SourceTasks and passed to Copycat for storage in + * Kafka. In addition to the standard fields in CopycatRecord which specify where data is stored + * in Kafka, they also include a sourcePartition and sourceOffset. + *

            + *

            + * The sourcePartition represents a single input sourcePartition that the record came from (e.g. a filename, table + * name, or topic-partition). The sourceOffset represents a position in that sourcePartition which can be used + * to resume consumption of data. + *

            + *

            + * These values can have arbitrary structure and should be represented using + * org.apache.kafka.copycat.data objects (or primitive values). For example, a database connector + * might specify the sourcePartition as a record containing { "db": "database_name", "table": + * "table_name"} and the sourceOffset as a Long containing the timestamp of the row. + *

            + */ +@InterfaceStability.Unstable +public class SourceRecord extends CopycatRecord { + private final Map sourcePartition; + private final Map sourceOffset; + + public SourceRecord(Map sourcePartition, Map sourceOffset, + String topic, Integer partition, Schema valueSchema, Object value) { + this(sourcePartition, sourceOffset, topic, partition, null, null, valueSchema, value); + } + + public SourceRecord(Map sourcePartition, Map sourceOffset, + String topic, Schema valueSchema, Object value) { + this(sourcePartition, sourceOffset, topic, null, null, null, valueSchema, value); + } + + public SourceRecord(Map sourcePartition, Map sourceOffset, + String topic, Integer partition, + Schema keySchema, Object key, Schema valueSchema, Object value) { + super(topic, partition, keySchema, key, valueSchema, value); + this.sourcePartition = sourcePartition; + this.sourceOffset = sourceOffset; + } + + public Map sourcePartition() { + return sourcePartition; + } + + public Map sourceOffset() { + return sourceOffset; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + if (!super.equals(o)) + return false; + + SourceRecord that = (SourceRecord) o; + + if (sourcePartition != null ? !sourcePartition.equals(that.sourcePartition) : that.sourcePartition != null) + return false; + if (sourceOffset != null ? !sourceOffset.equals(that.sourceOffset) : that.sourceOffset != null) + return false; + + return true; + } + + @Override + public int hashCode() { + int result = super.hashCode(); + result = 31 * result + (sourcePartition != null ? sourcePartition.hashCode() : 0); + result = 31 * result + (sourceOffset != null ? sourceOffset.hashCode() : 0); + return result; + } + + @Override + public String toString() { + return "SourceRecord{" + + "sourcePartition=" + sourcePartition + + ", sourceOffset=" + sourceOffset + + "} " + super.toString(); + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceTask.java b/copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceTask.java new file mode 100644 index 0000000000000..1e1da345979b2 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceTask.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.copycat.source; + +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.copycat.connector.Task; + +import java.util.List; + +/** + * SourceTask is a Task that pulls records from another system for storage in Kafka. + */ +@InterfaceStability.Unstable +public abstract class SourceTask implements Task { + + protected SourceTaskContext context; + + /** + * Initialize this SourceTask with the specified context object. + */ + public void initialize(SourceTaskContext context) { + this.context = context; + } + + /** + * Poll this SourceTask for new records. This method should block if no data is currently + * available. + * + * @return a list of source records + */ + public abstract List poll() throws InterruptedException; + + /** + *

            + * Commit the offsets, up to the offsets that have been returned by {@link #poll()}. This + * method should block until the commit is complete. + *

            + *

            + * SourceTasks are not required to implement this functionality; Copycat will record offsets + * automatically. This hook is provided for systems that also need to store offsets internally + * in their own system. + *

            + */ + public void commit() throws InterruptedException { + // This space intentionally left blank. + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceTaskContext.java b/copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceTaskContext.java new file mode 100644 index 0000000000000..a3875e75ab30e --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceTaskContext.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.copycat.source; + +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.copycat.storage.OffsetStorageReader; + +/** + * SourceTaskContext is provided to SourceTasks to allow them to interact with the underlying + * runtime. + */ +@InterfaceStability.Unstable +public class SourceTaskContext { + private final OffsetStorageReader reader; + + public SourceTaskContext(OffsetStorageReader reader) { + this.reader = reader; + } + + /** + * Get the OffsetStorageReader for this SourceTask. + */ + public OffsetStorageReader offsetStorageReader() { + return reader; + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/storage/Converter.java b/copycat/api/src/main/java/org/apache/kafka/copycat/storage/Converter.java new file mode 100644 index 0000000000000..d51b789db3e87 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/storage/Converter.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.copycat.storage; + +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.copycat.data.Schema; +import org.apache.kafka.copycat.data.SchemaAndValue; + +import java.util.Map; + +/** + * The Converter interface provides support for translating between Copycat's runtime data format + * and byte[]. Internally, this likely includes an intermediate step to the format used by the serialization + * layer (e.g. JsonNode, GenericRecord, Message). + */ +@InterfaceStability.Unstable +public interface Converter { + + /** + * Configure this class. + * @param configs configs in key/value pairs + * @param isKey whether is for key or value + */ + void configure(Map configs, boolean isKey); + + /** + * Convert a Copycat data object to a native object for serialization. + * @param topic the topic associated with the data + * @param schema the schema for the value + * @param value the value to convert + * @return + */ + byte[] fromCopycatData(String topic, Schema schema, Object value); + + /** + * Convert a native object to a Copycat data object. + * @param topic the topic associated with the data + * @param value the value to convert + * @return an object containing the {@link Schema} and the converted value + */ + SchemaAndValue toCopycatData(String topic, byte[] value); +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageReader.java b/copycat/api/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageReader.java new file mode 100644 index 0000000000000..95d2c04e32d85 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageReader.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.copycat.storage; + +import org.apache.kafka.common.annotation.InterfaceStability; + +import java.util.Collection; +import java.util.Map; + +/** + *

            + * OffsetStorageReader provides access to the offset storage used by sources. This can be used by + * connectors to determine offsets to start consuming data from. This is most commonly used during + * initialization of a task, but can also be used during runtime, e.g. when reconfiguring a task. + *

            + *

            + * Offsets are always defined as Maps of Strings to primitive types, i.e. all types supported by + * {@link org.apache.kafka.copycat.data.Schema} other than Array, Map, and Struct. + *

            + */ +@InterfaceStability.Unstable +public interface OffsetStorageReader { + /** + * Get the offset for the specified partition. If the data isn't already available locally, this + * gets it from the backing store, which may require some network round trips. + * + * @param partition object uniquely identifying the partition of data + * @return object uniquely identifying the offset in the partition of data + */ + Map offset(Map partition); + + /** + *

            + * Get a set of offsets for the specified partition identifiers. This may be more efficient + * than calling {@link #offset(Map)} repeatedly. + *

            + *

            + * Note that when errors occur, this method omits the associated data and tries to return as + * many of the requested values as possible. This allows a task that's managing many partitions to + * still proceed with any available data. Therefore, implementations should take care to check + * that the data is actually available in the returned response. The only case when an + * exception will be thrown is if the entire request failed, e.g. because the underlying + * storage was unavailable. + *

            + * + * @param partitions set of identifiers for partitions of data + * @return a map of partition identifiers to decoded offsets + */ + Map, Map> offsets(Collection> partitions); +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/storage/StringConverter.java b/copycat/api/src/main/java/org/apache/kafka/copycat/storage/StringConverter.java new file mode 100644 index 0000000000000..8d708f81e834b --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/storage/StringConverter.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.copycat.storage; + +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.copycat.data.Schema; +import org.apache.kafka.copycat.data.SchemaAndValue; +import org.apache.kafka.copycat.errors.DataException; + +import java.util.HashMap; +import java.util.Map; + +/** + * {@link Converter} implementation that only supports serializing to strings. When converting Copycat data to bytes, + * the schema will be ignored and {@link Object#toString()} will always be invoked to convert the data to a String. + * When converting from bytes to Copycat format, the converter will only ever return an optional string schema and + * a string or null. + * + * Encoding configuration is identical to {@link StringSerializer} and {@link StringDeserializer}, but for convenience + * this class can also be configured to use the same encoding for both encoding and decoding with the converter.encoding + * setting. + */ +public class StringConverter implements Converter { + private final StringSerializer serializer = new StringSerializer(); + private final StringDeserializer deserializer = new StringDeserializer(); + + public StringConverter() { + } + + @Override + public void configure(Map configs, boolean isKey) { + Map serializerConfigs = new HashMap<>(); + serializerConfigs.putAll(configs); + Map deserializerConfigs = new HashMap<>(); + deserializerConfigs.putAll(configs); + + Object encodingValue = configs.get("converter.encoding"); + if (encodingValue != null) { + serializerConfigs.put("serializer.encoding", encodingValue); + deserializerConfigs.put("deserializer.encoding", encodingValue); + } + + serializer.configure(serializerConfigs, isKey); + deserializer.configure(deserializerConfigs, isKey); + } + + @Override + public byte[] fromCopycatData(String topic, Schema schema, Object value) { + try { + return serializer.serialize(topic, value == null ? null : value.toString()); + } catch (SerializationException e) { + throw new DataException("Failed to serialize to a string: ", e); + } + } + + @Override + public SchemaAndValue toCopycatData(String topic, byte[] value) { + try { + return new SchemaAndValue(Schema.OPTIONAL_STRING_SCHEMA, deserializer.deserialize(topic, value)); + } catch (SerializationException e) { + throw new DataException("Failed to deserialize string: ", e); + } + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/util/ConnectorUtils.java b/copycat/api/src/main/java/org/apache/kafka/copycat/util/ConnectorUtils.java new file mode 100644 index 0000000000000..f9dd53ad4a1b9 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/util/ConnectorUtils.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.copycat.util; + +import org.apache.kafka.common.annotation.InterfaceStability; + +import java.util.ArrayList; +import java.util.List; + +/** + * Utilities that connector implementations might find useful. Contains common building blocks + * for writing connectors. + */ +@InterfaceStability.Unstable +public class ConnectorUtils { + /** + * Given a list of elements and a target number of groups, generates list of groups of + * elements to match the target number of groups, spreading them evenly among the groups. + * This generates groups with contiguous elements, which results in intuitive ordering if + * your elements are also ordered (e.g. alphabetical lists of table names if you sort + * table names alphabetically to generate the raw partitions) or can result in efficient + * partitioning if elements are sorted according to some criteria that affects performance + * (e.g. topic partitions with the same leader). + * + * @param elements list of elements to partition + * @param numGroups the number of output groups to generate. + */ + public static List> groupPartitions(List elements, int numGroups) { + if (numGroups <= 0) + throw new IllegalArgumentException("Number of groups must be positive."); + + List> result = new ArrayList<>(numGroups); + + // Each group has either n+1 or n raw partitions + int perGroup = elements.size() / numGroups; + int leftover = elements.size() - (numGroups * perGroup); + + int assigned = 0; + for (int group = 0; group < numGroups; group++) { + int numThisGroup = group < leftover ? perGroup + 1 : perGroup; + List groupList = new ArrayList<>(numThisGroup); + for (int i = 0; i < numThisGroup; i++) { + groupList.add(elements.get(assigned)); + assigned++; + } + result.add(groupList); + } + + return result; + } +} diff --git a/copycat/api/src/test/java/org/apache/kafka/copycat/connector/ConnectorReconfigurationTest.java b/copycat/api/src/test/java/org/apache/kafka/copycat/connector/ConnectorReconfigurationTest.java new file mode 100644 index 0000000000000..cbaf86610bf6c --- /dev/null +++ b/copycat/api/src/test/java/org/apache/kafka/copycat/connector/ConnectorReconfigurationTest.java @@ -0,0 +1,76 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.connector; + +import org.apache.kafka.copycat.errors.CopycatException; +import org.junit.Test; + +import java.util.List; +import java.util.Properties; + +import static org.junit.Assert.assertEquals; + +public class ConnectorReconfigurationTest { + + @Test + public void testDefaultReconfigure() throws Exception { + TestConnector conn = new TestConnector(false); + conn.reconfigure(new Properties()); + assertEquals(conn.stopOrder, 0); + assertEquals(conn.configureOrder, 1); + } + + @Test(expected = CopycatException.class) + public void testReconfigureStopException() throws Exception { + TestConnector conn = new TestConnector(true); + conn.reconfigure(new Properties()); + } + + private static class TestConnector extends Connector { + private boolean stopException; + private int order = 0; + public int stopOrder = -1; + public int configureOrder = -1; + + public TestConnector(boolean stopException) { + this.stopException = stopException; + } + + @Override + public void start(Properties props) { + configureOrder = order++; + } + + @Override + public Class taskClass() { + return null; + } + + @Override + public List taskConfigs(int count) { + return null; + } + + @Override + public void stop() { + stopOrder = order++; + if (stopException) + throw new CopycatException("error"); + } + } +} \ No newline at end of file diff --git a/copycat/api/src/test/java/org/apache/kafka/copycat/data/CopycatSchemaTest.java b/copycat/api/src/test/java/org/apache/kafka/copycat/data/CopycatSchemaTest.java new file mode 100644 index 0000000000000..49769509a21f0 --- /dev/null +++ b/copycat/api/src/test/java/org/apache/kafka/copycat/data/CopycatSchemaTest.java @@ -0,0 +1,303 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.data; + +import org.apache.kafka.copycat.errors.DataException; +import org.junit.Test; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.nio.CharBuffer; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNull; + +public class CopycatSchemaTest { + private static final Schema MAP_INT_STRING_SCHEMA = SchemaBuilder.map(Schema.INT32_SCHEMA, Schema.STRING_SCHEMA).build(); + private static final Schema FLAT_STRUCT_SCHEMA = SchemaBuilder.struct() + .field("field", Schema.INT32_SCHEMA) + .build(); + private static final Schema STRUCT_SCHEMA = SchemaBuilder.struct() + .field("first", Schema.INT32_SCHEMA) + .field("second", Schema.STRING_SCHEMA) + .field("array", SchemaBuilder.array(Schema.INT32_SCHEMA).build()) + .field("map", SchemaBuilder.map(Schema.INT32_SCHEMA, Schema.STRING_SCHEMA).build()) + .field("nested", FLAT_STRUCT_SCHEMA) + .build(); + private static final Schema PARENT_STRUCT_SCHEMA = SchemaBuilder.struct() + .field("nested", FLAT_STRUCT_SCHEMA) + .build(); + + @Test + public void testFieldsOnStructSchema() { + Schema schema = SchemaBuilder.struct() + .field("foo", Schema.BOOLEAN_SCHEMA) + .field("bar", Schema.INT32_SCHEMA) + .build(); + + assertEquals(2, schema.fields().size()); + // Validate field lookup by name + Field foo = schema.field("foo"); + assertEquals(0, foo.index()); + Field bar = schema.field("bar"); + assertEquals(1, bar.index()); + // Any other field name should fail + assertNull(schema.field("other")); + } + + + @Test(expected = DataException.class) + public void testFieldsOnlyValidForStructs() { + Schema.INT8_SCHEMA.fields(); + } + + @Test + public void testValidateValueMatchingType() { + CopycatSchema.validateValue(Schema.INT8_SCHEMA, (byte) 1); + CopycatSchema.validateValue(Schema.INT16_SCHEMA, (short) 1); + CopycatSchema.validateValue(Schema.INT32_SCHEMA, 1); + CopycatSchema.validateValue(Schema.INT64_SCHEMA, (long) 1); + CopycatSchema.validateValue(Schema.FLOAT32_SCHEMA, 1.f); + CopycatSchema.validateValue(Schema.FLOAT64_SCHEMA, 1.); + CopycatSchema.validateValue(Schema.BOOLEAN_SCHEMA, true); + CopycatSchema.validateValue(Schema.STRING_SCHEMA, "a string"); + CopycatSchema.validateValue(Schema.BYTES_SCHEMA, "a byte array".getBytes()); + CopycatSchema.validateValue(Schema.BYTES_SCHEMA, ByteBuffer.wrap("a byte array".getBytes())); + CopycatSchema.validateValue(SchemaBuilder.array(Schema.INT32_SCHEMA).build(), Arrays.asList(1, 2, 3)); + CopycatSchema.validateValue( + SchemaBuilder.map(Schema.INT32_SCHEMA, Schema.STRING_SCHEMA).build(), + Collections.singletonMap(1, "value") + ); + // Struct tests the basic struct layout + complex field types + nested structs + Struct structValue = new Struct(STRUCT_SCHEMA) + .put("first", 1) + .put("second", "foo") + .put("array", Arrays.asList(1, 2, 3)) + .put("map", Collections.singletonMap(1, "value")) + .put("nested", new Struct(FLAT_STRUCT_SCHEMA).put("field", 12)); + CopycatSchema.validateValue(STRUCT_SCHEMA, structValue); + } + + @Test + public void testValidateValueMatchingLogicalType() { + CopycatSchema.validateValue(Decimal.schema(2), new BigDecimal(new BigInteger("156"), 2)); + CopycatSchema.validateValue(Date.SCHEMA, new java.util.Date(0)); + CopycatSchema.validateValue(Time.SCHEMA, new java.util.Date(0)); + CopycatSchema.validateValue(Timestamp.SCHEMA, new java.util.Date(0)); + } + + // To avoid requiring excessive numbers of tests, these checks for invalid types use a similar type where possible + // to only include a single test for each type + + @Test(expected = DataException.class) + public void testValidateValueMismatchInt8() { + CopycatSchema.validateValue(Schema.INT8_SCHEMA, 1); + } + + @Test(expected = DataException.class) + public void testValidateValueMismatchInt16() { + CopycatSchema.validateValue(Schema.INT16_SCHEMA, 1); + } + + @Test(expected = DataException.class) + public void testValidateValueMismatchInt32() { + CopycatSchema.validateValue(Schema.INT32_SCHEMA, (long) 1); + } + + @Test(expected = DataException.class) + public void testValidateValueMismatchInt64() { + CopycatSchema.validateValue(Schema.INT64_SCHEMA, 1); + } + + @Test(expected = DataException.class) + public void testValidateValueMismatchFloat() { + CopycatSchema.validateValue(Schema.FLOAT32_SCHEMA, 1.0); + } + + @Test(expected = DataException.class) + public void testValidateValueMismatchDouble() { + CopycatSchema.validateValue(Schema.FLOAT64_SCHEMA, 1.f); + } + + @Test(expected = DataException.class) + public void testValidateValueMismatchBoolean() { + CopycatSchema.validateValue(Schema.BOOLEAN_SCHEMA, 1.f); + } + + @Test(expected = DataException.class) + public void testValidateValueMismatchString() { + // CharSequence is a similar type (supertype of String), but we restrict to String. + CharBuffer cbuf = CharBuffer.wrap("abc"); + CopycatSchema.validateValue(Schema.STRING_SCHEMA, cbuf); + } + + @Test(expected = DataException.class) + public void testValidateValueMismatchBytes() { + CopycatSchema.validateValue(Schema.BYTES_SCHEMA, new Object[]{1, "foo"}); + } + + @Test(expected = DataException.class) + public void testValidateValueMismatchArray() { + CopycatSchema.validateValue(SchemaBuilder.array(Schema.INT32_SCHEMA).build(), Arrays.asList("a", "b", "c")); + } + + @Test(expected = DataException.class) + public void testValidateValueMismatchArraySomeMatch() { + // Even if some match the right type, this should fail if any mismatch. In this case, type erasure loses + // the fact that the list is actually List, but we couldn't tell if only checking the first element + CopycatSchema.validateValue(SchemaBuilder.array(Schema.INT32_SCHEMA).build(), Arrays.asList(1, 2, "c")); + } + + @Test(expected = DataException.class) + public void testValidateValueMismatchMapKey() { + CopycatSchema.validateValue(MAP_INT_STRING_SCHEMA, Collections.singletonMap("wrong key type", "value")); + } + + @Test(expected = DataException.class) + public void testValidateValueMismatchMapValue() { + CopycatSchema.validateValue(MAP_INT_STRING_SCHEMA, Collections.singletonMap(1, 2)); + } + + @Test(expected = DataException.class) + public void testValidateValueMismatchMapSomeKeys() { + Map data = new HashMap<>(); + data.put(1, "abc"); + data.put("wrong", "it's as easy as one two three"); + CopycatSchema.validateValue(MAP_INT_STRING_SCHEMA, data); + } + + @Test(expected = DataException.class) + public void testValidateValueMismatchMapSomeValues() { + Map data = new HashMap<>(); + data.put(1, "abc"); + data.put(2, "wrong".getBytes()); + CopycatSchema.validateValue(MAP_INT_STRING_SCHEMA, data); + } + + @Test(expected = DataException.class) + public void testValidateValueMismatchStructWrongSchema() { + // Completely mismatching schemas + CopycatSchema.validateValue( + FLAT_STRUCT_SCHEMA, + new Struct(SchemaBuilder.struct().field("x", Schema.INT32_SCHEMA).build()).put("x", 1) + ); + } + + @Test(expected = DataException.class) + public void testValidateValueMismatchStructWrongNestedSchema() { + // Top-level schema matches, but nested does not. + CopycatSchema.validateValue( + PARENT_STRUCT_SCHEMA, + new Struct(PARENT_STRUCT_SCHEMA) + .put("nested", new Struct(SchemaBuilder.struct().field("x", Schema.INT32_SCHEMA).build()).put("x", 1)) + ); + } + + @Test(expected = DataException.class) + public void testValidateValueMismatchDecimal() { + CopycatSchema.validateValue(Decimal.schema(2), new BigInteger("156")); + } + + @Test(expected = DataException.class) + public void testValidateValueMismatchDate() { + CopycatSchema.validateValue(Date.SCHEMA, 1000L); + } + + @Test(expected = DataException.class) + public void testValidateValueMismatchTime() { + CopycatSchema.validateValue(Time.SCHEMA, 1000L); + } + + @Test(expected = DataException.class) + public void testValidateValueMismatchTimestamp() { + CopycatSchema.validateValue(Timestamp.SCHEMA, 1000L); + } + + @Test + public void testPrimitiveEquality() { + // Test that primitive types, which only need to consider all the type & metadata fields, handle equality correctly + CopycatSchema s1 = new CopycatSchema(Schema.Type.INT8, false, null, "name", 2, "doc"); + CopycatSchema s2 = new CopycatSchema(Schema.Type.INT8, false, null, "name", 2, "doc"); + CopycatSchema differentType = new CopycatSchema(Schema.Type.INT16, false, null, "name", 2, "doc"); + CopycatSchema differentOptional = new CopycatSchema(Schema.Type.INT8, true, null, "name", 2, "doc"); + CopycatSchema differentDefault = new CopycatSchema(Schema.Type.INT8, false, true, "name", 2, "doc"); + CopycatSchema differentName = new CopycatSchema(Schema.Type.INT8, false, null, "otherName", 2, "doc"); + CopycatSchema differentVersion = new CopycatSchema(Schema.Type.INT8, false, null, "name", 4, "doc"); + CopycatSchema differentDoc = new CopycatSchema(Schema.Type.INT8, false, null, "name", 2, "other doc"); + CopycatSchema differentParameters = new CopycatSchema(Schema.Type.INT8, false, null, "name", 2, "doc", Collections.singletonMap("param", "value"), null, null, null); + + assertEquals(s1, s2); + assertNotEquals(s1, differentType); + assertNotEquals(s1, differentOptional); + assertNotEquals(s1, differentDefault); + assertNotEquals(s1, differentName); + assertNotEquals(s1, differentVersion); + assertNotEquals(s1, differentDoc); + assertNotEquals(s1, differentParameters); + } + + @Test + public void testArrayEquality() { + // Validate that the value type for the array is tested for equality. This test makes sure the same schema object is + // never reused to ensure we're actually checking equality + CopycatSchema s1 = new CopycatSchema(Schema.Type.ARRAY, false, null, null, null, null, null, null, null, SchemaBuilder.int8().build()); + CopycatSchema s2 = new CopycatSchema(Schema.Type.ARRAY, false, null, null, null, null, null, null, null, SchemaBuilder.int8().build()); + CopycatSchema differentValueSchema = new CopycatSchema(Schema.Type.ARRAY, false, null, null, null, null, null, null, null, SchemaBuilder.int16().build()); + + assertEquals(s1, s2); + assertNotEquals(s1, differentValueSchema); + } + + @Test + public void testMapEquality() { + // Same as testArrayEquality, but for both key and value schemas + CopycatSchema s1 = new CopycatSchema(Schema.Type.MAP, false, null, null, null, null, null, null, SchemaBuilder.int8().build(), SchemaBuilder.int16().build()); + CopycatSchema s2 = new CopycatSchema(Schema.Type.MAP, false, null, null, null, null, null, null, SchemaBuilder.int8().build(), SchemaBuilder.int16().build()); + CopycatSchema differentKeySchema = new CopycatSchema(Schema.Type.MAP, false, null, null, null, null, null, null, SchemaBuilder.string().build(), SchemaBuilder.int16().build()); + CopycatSchema differentValueSchema = new CopycatSchema(Schema.Type.MAP, false, null, null, null, null, null, null, SchemaBuilder.int8().build(), SchemaBuilder.string().build()); + + assertEquals(s1, s2); + assertNotEquals(s1, differentKeySchema); + assertNotEquals(s1, differentValueSchema); + } + + @Test + public void testStructEquality() { + // Same as testArrayEquality, but checks differences in fields. Only does a simple check, relying on tests of + // Field's equals() method to validate all variations in the list of fields will be checked + CopycatSchema s1 = new CopycatSchema(Schema.Type.STRUCT, false, null, null, null, null, null, + Arrays.asList(new Field("field", 0, SchemaBuilder.int8().build()), + new Field("field2", 1, SchemaBuilder.int16().build())), null, null); + CopycatSchema s2 = new CopycatSchema(Schema.Type.STRUCT, false, null, null, null, null, null, + Arrays.asList(new Field("field", 0, SchemaBuilder.int8().build()), + new Field("field2", 1, SchemaBuilder.int16().build())), null, null); + CopycatSchema differentField = new CopycatSchema(Schema.Type.STRUCT, false, null, null, null, null, null, + Arrays.asList(new Field("field", 0, SchemaBuilder.int8().build()), + new Field("different field name", 1, SchemaBuilder.int16().build())), null, null); + + assertEquals(s1, s2); + assertNotEquals(s1, differentField); + } + +} diff --git a/copycat/api/src/test/java/org/apache/kafka/copycat/data/DateTest.java b/copycat/api/src/test/java/org/apache/kafka/copycat/data/DateTest.java new file mode 100644 index 0000000000000..e7885ab98627f --- /dev/null +++ b/copycat/api/src/test/java/org/apache/kafka/copycat/data/DateTest.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.copycat.data; + +import org.apache.kafka.copycat.errors.DataException; +import org.junit.Test; + +import java.util.Calendar; +import java.util.GregorianCalendar; +import java.util.TimeZone; + +import static org.junit.Assert.assertEquals; + +public class DateTest { + private static final GregorianCalendar EPOCH; + private static final GregorianCalendar EPOCH_PLUS_TEN_THOUSAND_DAYS; + private static final GregorianCalendar EPOCH_PLUS_TIME_COMPONENT; + static { + EPOCH = new GregorianCalendar(1970, Calendar.JANUARY, 1, 0, 0, 0); + EPOCH.setTimeZone(TimeZone.getTimeZone("UTC")); + + EPOCH_PLUS_TIME_COMPONENT = new GregorianCalendar(1970, Calendar.JANUARY, 1, 0, 0, 1); + EPOCH_PLUS_TIME_COMPONENT.setTimeZone(TimeZone.getTimeZone("UTC")); + + EPOCH_PLUS_TEN_THOUSAND_DAYS = new GregorianCalendar(1970, Calendar.JANUARY, 1, 0, 0, 0); + EPOCH_PLUS_TEN_THOUSAND_DAYS.setTimeZone(TimeZone.getTimeZone("UTC")); + EPOCH_PLUS_TEN_THOUSAND_DAYS.add(Calendar.DATE, 10000); + } + + @Test + public void testBuilder() { + Schema plain = Date.SCHEMA; + assertEquals(Date.LOGICAL_NAME, plain.name()); + assertEquals(1, (Object) plain.version()); + } + + @Test + public void testFromLogical() { + assertEquals(0, Date.fromLogical(Date.SCHEMA, EPOCH.getTime())); + assertEquals(10000, Date.fromLogical(Date.SCHEMA, EPOCH_PLUS_TEN_THOUSAND_DAYS.getTime())); + } + + @Test(expected = DataException.class) + public void testFromLogicalInvalidSchema() { + Date.fromLogical(Date.builder().name("invalid").build(), EPOCH.getTime()); + } + + @Test(expected = DataException.class) + public void testFromLogicalInvalidHasTimeComponents() { + Date.fromLogical(Date.SCHEMA, EPOCH_PLUS_TIME_COMPONENT.getTime()); + } + + @Test + public void testToLogical() { + assertEquals(EPOCH.getTime(), Date.toLogical(Date.SCHEMA, 0)); + assertEquals(EPOCH_PLUS_TEN_THOUSAND_DAYS.getTime(), Date.toLogical(Date.SCHEMA, 10000)); + } + + @Test(expected = DataException.class) + public void testToLogicalInvalidSchema() { + Date.toLogical(Date.builder().name("invalid").build(), 0); + } +} diff --git a/copycat/api/src/test/java/org/apache/kafka/copycat/data/DecimalTest.java b/copycat/api/src/test/java/org/apache/kafka/copycat/data/DecimalTest.java new file mode 100644 index 0000000000000..ce711618e14e3 --- /dev/null +++ b/copycat/api/src/test/java/org/apache/kafka/copycat/data/DecimalTest.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.copycat.data; + +import org.junit.Test; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.util.Collections; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +public class DecimalTest { + private static final int TEST_SCALE = 2; + private static final BigDecimal TEST_DECIMAL = new BigDecimal(new BigInteger("156"), TEST_SCALE); + private static final BigDecimal TEST_DECIMAL_NEGATIVE = new BigDecimal(new BigInteger("-156"), TEST_SCALE); + private static final byte[] TEST_BYTES = new byte[]{0, -100}; + private static final byte[] TEST_BYTES_NEGATIVE = new byte[]{-1, 100}; + + @Test + public void testBuilder() { + Schema plain = Decimal.builder(2).build(); + assertEquals(Decimal.LOGICAL_NAME, plain.name()); + assertEquals(Collections.singletonMap(Decimal.SCALE_FIELD, "2"), plain.parameters()); + assertEquals(1, (Object) plain.version()); + } + + @Test + public void testFromLogical() { + Schema schema = Decimal.schema(TEST_SCALE); + byte[] encoded = Decimal.fromLogical(schema, TEST_DECIMAL); + assertArrayEquals(TEST_BYTES, encoded); + + encoded = Decimal.fromLogical(schema, TEST_DECIMAL_NEGATIVE); + assertArrayEquals(TEST_BYTES_NEGATIVE, encoded); + } + + @Test + public void testToLogical() { + Schema schema = Decimal.schema(2); + BigDecimal converted = Decimal.toLogical(schema, TEST_BYTES); + assertEquals(TEST_DECIMAL, converted); + + converted = Decimal.toLogical(schema, TEST_BYTES_NEGATIVE); + assertEquals(TEST_DECIMAL_NEGATIVE, converted); + } +} diff --git a/copycat/api/src/test/java/org/apache/kafka/copycat/data/FieldTest.java b/copycat/api/src/test/java/org/apache/kafka/copycat/data/FieldTest.java new file mode 100644 index 0000000000000..d5458bc59be46 --- /dev/null +++ b/copycat/api/src/test/java/org/apache/kafka/copycat/data/FieldTest.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.copycat.data; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +public class FieldTest { + + @Test + public void testEquality() { + Field field1 = new Field("name", 0, Schema.INT8_SCHEMA); + Field field2 = new Field("name", 0, Schema.INT8_SCHEMA); + Field differentName = new Field("name2", 0, Schema.INT8_SCHEMA); + Field differentIndex = new Field("name", 1, Schema.INT8_SCHEMA); + Field differentSchema = new Field("name", 0, Schema.INT16_SCHEMA); + + assertEquals(field1, field2); + assertNotEquals(field1, differentName); + assertNotEquals(field1, differentIndex); + assertNotEquals(field1, differentSchema); + } +} diff --git a/copycat/api/src/test/java/org/apache/kafka/copycat/data/SchemaBuilderTest.java b/copycat/api/src/test/java/org/apache/kafka/copycat/data/SchemaBuilderTest.java new file mode 100644 index 0000000000000..183f5fca7f3d1 --- /dev/null +++ b/copycat/api/src/test/java/org/apache/kafka/copycat/data/SchemaBuilderTest.java @@ -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 org.apache.kafka.copycat.data; + +import org.apache.kafka.copycat.errors.SchemaBuilderException; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +public class SchemaBuilderTest { + private static final String NAME = "name"; + private static final Integer VERSION = 2; + private static final String DOC = "doc"; + private static final Map NO_PARAMS = null; + + @Test + public void testInt8Builder() { + Schema schema = SchemaBuilder.int8().build(); + assertTypeAndDefault(schema, Schema.Type.INT8, false, null); + assertNoMetadata(schema); + + schema = SchemaBuilder.int8().name(NAME).optional().defaultValue((byte) 12) + .version(VERSION).doc(DOC).build(); + assertTypeAndDefault(schema, Schema.Type.INT8, true, (byte) 12); + assertMetadata(schema, NAME, VERSION, DOC, NO_PARAMS); + } + + @Test(expected = SchemaBuilderException.class) + public void testInt8BuilderInvalidDefault() { + SchemaBuilder.int8().defaultValue("invalid"); + } + + @Test + public void testInt16Builder() { + Schema schema = SchemaBuilder.int16().build(); + assertTypeAndDefault(schema, Schema.Type.INT16, false, null); + assertNoMetadata(schema); + + schema = SchemaBuilder.int16().name(NAME).optional().defaultValue((short) 12) + .version(VERSION).doc(DOC).build(); + assertTypeAndDefault(schema, Schema.Type.INT16, true, (short) 12); + assertMetadata(schema, NAME, VERSION, DOC, NO_PARAMS); + } + + @Test(expected = SchemaBuilderException.class) + public void testInt16BuilderInvalidDefault() { + SchemaBuilder.int16().defaultValue("invalid"); + } + + @Test + public void testInt32Builder() { + Schema schema = SchemaBuilder.int32().build(); + assertTypeAndDefault(schema, Schema.Type.INT32, false, null); + assertNoMetadata(schema); + + schema = SchemaBuilder.int32().name(NAME).optional().defaultValue(12) + .version(VERSION).doc(DOC).build(); + assertTypeAndDefault(schema, Schema.Type.INT32, true, 12); + assertMetadata(schema, NAME, VERSION, DOC, NO_PARAMS); + } + + @Test(expected = SchemaBuilderException.class) + public void testInt32BuilderInvalidDefault() { + SchemaBuilder.int32().defaultValue("invalid"); + } + + @Test + public void testInt64Builder() { + Schema schema = SchemaBuilder.int64().build(); + assertTypeAndDefault(schema, Schema.Type.INT64, false, null); + assertNoMetadata(schema); + + schema = SchemaBuilder.int64().name(NAME).optional().defaultValue((long) 12) + .version(VERSION).doc(DOC).build(); + assertTypeAndDefault(schema, Schema.Type.INT64, true, (long) 12); + assertMetadata(schema, NAME, VERSION, DOC, NO_PARAMS); + } + + @Test(expected = SchemaBuilderException.class) + public void testInt64BuilderInvalidDefault() { + SchemaBuilder.int64().defaultValue("invalid"); + } + + @Test + public void testFloatBuilder() { + Schema schema = SchemaBuilder.float32().build(); + assertTypeAndDefault(schema, Schema.Type.FLOAT32, false, null); + assertNoMetadata(schema); + + schema = SchemaBuilder.float32().name(NAME).optional().defaultValue(12.f) + .version(VERSION).doc(DOC).build(); + assertTypeAndDefault(schema, Schema.Type.FLOAT32, true, 12.f); + assertMetadata(schema, NAME, VERSION, DOC, NO_PARAMS); + } + + @Test(expected = SchemaBuilderException.class) + public void testFloatBuilderInvalidDefault() { + SchemaBuilder.float32().defaultValue("invalid"); + } + + @Test + public void testDoubleBuilder() { + Schema schema = SchemaBuilder.float64().build(); + assertTypeAndDefault(schema, Schema.Type.FLOAT64, false, null); + assertNoMetadata(schema); + + schema = SchemaBuilder.float64().name(NAME).optional().defaultValue(12.0) + .version(VERSION).doc(DOC).build(); + assertTypeAndDefault(schema, Schema.Type.FLOAT64, true, 12.0); + assertMetadata(schema, NAME, VERSION, DOC, NO_PARAMS); + } + + @Test(expected = SchemaBuilderException.class) + public void testDoubleBuilderInvalidDefault() { + SchemaBuilder.float64().defaultValue("invalid"); + } + + @Test + public void testBooleanBuilder() { + Schema schema = SchemaBuilder.bool().build(); + assertTypeAndDefault(schema, Schema.Type.BOOLEAN, false, null); + assertNoMetadata(schema); + + schema = SchemaBuilder.bool().name(NAME).optional().defaultValue(true) + .version(VERSION).doc(DOC).build(); + assertTypeAndDefault(schema, Schema.Type.BOOLEAN, true, true); + assertMetadata(schema, NAME, VERSION, DOC, NO_PARAMS); + } + + @Test(expected = SchemaBuilderException.class) + public void testBooleanBuilderInvalidDefault() { + SchemaBuilder.bool().defaultValue("invalid"); + } + + @Test + public void testStringBuilder() { + Schema schema = SchemaBuilder.string().build(); + assertTypeAndDefault(schema, Schema.Type.STRING, false, null); + assertNoMetadata(schema); + + schema = SchemaBuilder.string().name(NAME).optional().defaultValue("a default string") + .version(VERSION).doc(DOC).build(); + assertTypeAndDefault(schema, Schema.Type.STRING, true, "a default string"); + assertMetadata(schema, NAME, VERSION, DOC, NO_PARAMS); + } + + @Test(expected = SchemaBuilderException.class) + public void testStringBuilderInvalidDefault() { + SchemaBuilder.string().defaultValue(true); + } + + @Test + public void testBytesBuilder() { + Schema schema = SchemaBuilder.bytes().build(); + assertTypeAndDefault(schema, Schema.Type.BYTES, false, null); + assertNoMetadata(schema); + + schema = SchemaBuilder.bytes().name(NAME).optional().defaultValue("a default byte array".getBytes()) + .version(VERSION).doc(DOC).build(); + assertTypeAndDefault(schema, Schema.Type.BYTES, true, "a default byte array".getBytes()); + assertMetadata(schema, NAME, VERSION, DOC, NO_PARAMS); + } + + @Test(expected = SchemaBuilderException.class) + public void testBytesBuilderInvalidDefault() { + SchemaBuilder.bytes().defaultValue("a string, not bytes"); + } + + + @Test + public void testParameters() { + Map expectedParameters = new HashMap<>(); + expectedParameters.put("foo", "val"); + expectedParameters.put("bar", "baz"); + + Schema schema = SchemaBuilder.string().parameter("foo", "val").parameter("bar", "baz").build(); + assertTypeAndDefault(schema, Schema.Type.STRING, false, null); + assertMetadata(schema, null, null, null, expectedParameters); + + schema = SchemaBuilder.string().parameters(expectedParameters).build(); + assertTypeAndDefault(schema, Schema.Type.STRING, false, null); + assertMetadata(schema, null, null, null, expectedParameters); + } + + + @Test + public void testStructBuilder() { + Schema schema = SchemaBuilder.struct() + .field("field1", Schema.INT8_SCHEMA) + .field("field2", Schema.INT8_SCHEMA) + .build(); + assertTypeAndDefault(schema, Schema.Type.STRUCT, false, null); + assertEquals(2, schema.fields().size()); + assertEquals("field1", schema.fields().get(0).name()); + assertEquals(0, schema.fields().get(0).index()); + assertEquals(Schema.INT8_SCHEMA, schema.fields().get(0).schema()); + assertEquals("field2", schema.fields().get(1).name()); + assertEquals(1, schema.fields().get(1).index()); + assertEquals(Schema.INT8_SCHEMA, schema.fields().get(1).schema()); + assertNoMetadata(schema); + } + + @Test(expected = SchemaBuilderException.class) + public void testNonStructCantHaveFields() { + SchemaBuilder.int8().field("field", SchemaBuilder.int8().build()); + } + + + @Test + public void testArrayBuilder() { + Schema schema = SchemaBuilder.array(Schema.INT8_SCHEMA).build(); + assertTypeAndDefault(schema, Schema.Type.ARRAY, false, null); + assertEquals(schema.valueSchema(), Schema.INT8_SCHEMA); + assertNoMetadata(schema); + + // Default value + List defArray = Arrays.asList((byte) 1, (byte) 2); + schema = SchemaBuilder.array(Schema.INT8_SCHEMA).defaultValue(defArray).build(); + assertTypeAndDefault(schema, Schema.Type.ARRAY, false, defArray); + assertEquals(schema.valueSchema(), Schema.INT8_SCHEMA); + assertNoMetadata(schema); + } + + @Test(expected = SchemaBuilderException.class) + public void testArrayBuilderInvalidDefault() { + // Array, but wrong embedded type + SchemaBuilder.array(Schema.INT8_SCHEMA).defaultValue(Arrays.asList("string")).build(); + } + + @Test + public void testMapBuilder() { + Schema schema = SchemaBuilder.map(Schema.INT8_SCHEMA, Schema.INT8_SCHEMA).build(); + assertTypeAndDefault(schema, Schema.Type.MAP, false, null); + assertEquals(schema.keySchema(), Schema.INT8_SCHEMA); + assertEquals(schema.valueSchema(), Schema.INT8_SCHEMA); + assertNoMetadata(schema); + + // Default value + Map defMap = Collections.singletonMap((byte) 5, (byte) 10); + schema = SchemaBuilder.map(Schema.INT8_SCHEMA, Schema.INT8_SCHEMA) + .defaultValue(defMap).build(); + assertTypeAndDefault(schema, Schema.Type.MAP, false, defMap); + assertEquals(schema.keySchema(), Schema.INT8_SCHEMA); + assertEquals(schema.valueSchema(), Schema.INT8_SCHEMA); + assertNoMetadata(schema); + } + + @Test(expected = SchemaBuilderException.class) + public void testMapBuilderInvalidDefault() { + // Map, but wrong embedded type + Map defMap = Collections.singletonMap((byte) 5, "foo"); + SchemaBuilder.map(Schema.INT8_SCHEMA, Schema.INT8_SCHEMA) + .defaultValue(defMap).build(); + } + + + + private void assertTypeAndDefault(Schema schema, Schema.Type type, boolean optional, Object defaultValue) { + assertEquals(type, schema.type()); + assertEquals(optional, schema.isOptional()); + if (type == Schema.Type.BYTES) { + // byte[] is not comparable, need to wrap to check correctly + if (defaultValue == null) + assertNull(schema.defaultValue()); + else + assertEquals(ByteBuffer.wrap((byte[]) defaultValue), ByteBuffer.wrap((byte[]) schema.defaultValue())); + } else { + assertEquals(defaultValue, schema.defaultValue()); + } + } + + private void assertMetadata(Schema schema, String name, Integer version, String doc, Map parameters) { + assertEquals(name, schema.name()); + assertEquals(version, schema.version()); + assertEquals(doc, schema.doc()); + assertEquals(parameters, schema.parameters()); + } + + private void assertNoMetadata(Schema schema) { + assertMetadata(schema, null, null, null, null); + } +} diff --git a/copycat/api/src/test/java/org/apache/kafka/copycat/data/StructTest.java b/copycat/api/src/test/java/org/apache/kafka/copycat/data/StructTest.java new file mode 100644 index 0000000000000..162396b587d02 --- /dev/null +++ b/copycat/api/src/test/java/org/apache/kafka/copycat/data/StructTest.java @@ -0,0 +1,222 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.data; + +import org.apache.kafka.copycat.errors.DataException; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +public class StructTest { + + private static final Schema FLAT_STRUCT_SCHEMA = SchemaBuilder.struct() + .field("int8", Schema.INT8_SCHEMA) + .field("int16", Schema.INT16_SCHEMA) + .field("int32", Schema.INT32_SCHEMA) + .field("int64", Schema.INT64_SCHEMA) + .field("float32", Schema.FLOAT32_SCHEMA) + .field("float64", Schema.FLOAT64_SCHEMA) + .field("boolean", Schema.BOOLEAN_SCHEMA) + .field("string", Schema.STRING_SCHEMA) + .field("bytes", Schema.BYTES_SCHEMA) + .build(); + + private static final Schema ARRAY_SCHEMA = SchemaBuilder.array(Schema.INT8_SCHEMA).build(); + private static final Schema MAP_SCHEMA = SchemaBuilder.map( + Schema.INT32_SCHEMA, + Schema.STRING_SCHEMA + ).build(); + private static final Schema NESTED_CHILD_SCHEMA = SchemaBuilder.struct() + .field("int8", Schema.INT8_SCHEMA) + .build(); + private static final Schema NESTED_SCHEMA = SchemaBuilder.struct() + .field("array", ARRAY_SCHEMA) + .field("map", MAP_SCHEMA) + .field("nested", NESTED_CHILD_SCHEMA) + .build(); + + private static final Schema REQUIRED_FIELD_SCHEMA = Schema.INT8_SCHEMA; + private static final Schema OPTIONAL_FIELD_SCHEMA = SchemaBuilder.int8().optional().build(); + private static final Schema DEFAULT_FIELD_SCHEMA = SchemaBuilder.int8().defaultValue((byte) 0).build(); + + @Test + public void testFlatStruct() { + Struct struct = new Struct(FLAT_STRUCT_SCHEMA) + .put("int8", (byte) 12) + .put("int16", (short) 12) + .put("int32", 12) + .put("int64", (long) 12) + .put("float32", 12.f) + .put("float64", 12.) + .put("boolean", true) + .put("string", "foobar") + .put("bytes", "foobar".getBytes()); + + // Test equality, and also the type-specific getters + assertEquals((byte) 12, (byte) struct.getInt8("int8")); + assertEquals((short) 12, (short) struct.getInt16("int16")); + assertEquals(12, (int) struct.getInt32("int32")); + assertEquals((long) 12, (long) struct.getInt64("int64")); + assertEquals((Float) 12.f, struct.getFloat32("float32")); + assertEquals((Double) 12., struct.getFloat64("float64")); + assertEquals(true, struct.getBoolean("boolean")); + assertEquals("foobar", struct.getString("string")); + assertEquals(ByteBuffer.wrap("foobar".getBytes()), ByteBuffer.wrap(struct.getBytes("bytes"))); + + struct.validate(); + } + + @Test + public void testComplexStruct() { + List array = Arrays.asList((byte) 1, (byte) 2); + Map map = Collections.singletonMap(1, "string"); + Struct struct = new Struct(NESTED_SCHEMA) + .put("array", array) + .put("map", map) + .put("nested", new Struct(NESTED_CHILD_SCHEMA).put("int8", (byte) 12)); + + // Separate the call to get the array and map to validate the typed get methods work properly + List arrayExtracted = struct.getArray("array"); + assertEquals(array, arrayExtracted); + Map mapExtracted = struct.getMap("map"); + assertEquals(map, mapExtracted); + assertEquals((byte) 12, struct.getStruct("nested").get("int8")); + + struct.validate(); + } + + + // These don't test all the ways validation can fail, just one for each element. See more extensive validation + // tests in SchemaTest. These are meant to ensure that we are invoking the same code path and that we do deeper + // inspection than just checking the class of the object + + @Test(expected = DataException.class) + public void testInvalidFieldType() { + new Struct(FLAT_STRUCT_SCHEMA).put("int8", "should fail because this is a string, not int8"); + } + + @Test(expected = DataException.class) + public void testInvalidArrayFieldElements() { + new Struct(NESTED_SCHEMA).put("array", Arrays.asList("should fail since elements should be int8s")); + } + + @Test(expected = DataException.class) + public void testInvalidMapKeyElements() { + new Struct(NESTED_SCHEMA).put("map", Collections.singletonMap("should fail because keys should be int8s", (byte) 12)); + } + + @Test(expected = DataException.class) + public void testInvalidStructFieldSchema() { + new Struct(NESTED_SCHEMA).put("nested", new Struct(MAP_SCHEMA)); + } + + @Test(expected = DataException.class) + public void testInvalidStructFieldValue() { + new Struct(NESTED_SCHEMA).put("nested", new Struct(NESTED_CHILD_SCHEMA)); + } + + + @Test(expected = DataException.class) + public void testMissingFieldValidation() { + // Required int8 field + Schema schema = SchemaBuilder.struct().field("field", REQUIRED_FIELD_SCHEMA).build(); + Struct struct = new Struct(schema); + struct.validate(); + } + + @Test + public void testMissingOptionalFieldValidation() { + Schema schema = SchemaBuilder.struct().field("field", OPTIONAL_FIELD_SCHEMA).build(); + Struct struct = new Struct(schema); + struct.validate(); + } + + @Test + public void testMissingFieldWithDefaultValidation() { + Schema schema = SchemaBuilder.struct().field("field", DEFAULT_FIELD_SCHEMA).build(); + Struct struct = new Struct(schema); + struct.validate(); + } + + + @Test + public void testEquals() { + Struct struct1 = new Struct(FLAT_STRUCT_SCHEMA) + .put("int8", (byte) 12) + .put("int16", (short) 12) + .put("int32", 12) + .put("int64", (long) 12) + .put("float32", 12.f) + .put("float64", 12.) + .put("boolean", true) + .put("string", "foobar") + .put("bytes", ByteBuffer.wrap("foobar".getBytes())); + Struct struct2 = new Struct(FLAT_STRUCT_SCHEMA) + .put("int8", (byte) 12) + .put("int16", (short) 12) + .put("int32", 12) + .put("int64", (long) 12) + .put("float32", 12.f) + .put("float64", 12.) + .put("boolean", true) + .put("string", "foobar") + .put("bytes", ByteBuffer.wrap("foobar".getBytes())); + Struct struct3 = new Struct(FLAT_STRUCT_SCHEMA) + .put("int8", (byte) 12) + .put("int16", (short) 12) + .put("int32", 12) + .put("int64", (long) 12) + .put("float32", 12.f) + .put("float64", 12.) + .put("boolean", true) + .put("string", "mismatching string") + .put("bytes", ByteBuffer.wrap("foobar".getBytes())); + + assertEquals(struct1, struct2); + assertNotEquals(struct1, struct3); + + List array = Arrays.asList((byte) 1, (byte) 2); + Map map = Collections.singletonMap(1, "string"); + struct1 = new Struct(NESTED_SCHEMA) + .put("array", array) + .put("map", map) + .put("nested", new Struct(NESTED_CHILD_SCHEMA).put("int8", (byte) 12)); + List array2 = Arrays.asList((byte) 1, (byte) 2); + Map map2 = Collections.singletonMap(1, "string"); + struct2 = new Struct(NESTED_SCHEMA) + .put("array", array2) + .put("map", map2) + .put("nested", new Struct(NESTED_CHILD_SCHEMA).put("int8", (byte) 12)); + List array3 = Arrays.asList((byte) 1, (byte) 2, (byte) 3); + Map map3 = Collections.singletonMap(2, "string"); + struct3 = new Struct(NESTED_SCHEMA) + .put("array", array3) + .put("map", map3) + .put("nested", new Struct(NESTED_CHILD_SCHEMA).put("int8", (byte) 13)); + + assertEquals(struct1, struct2); + assertNotEquals(struct1, struct3); + } +} diff --git a/copycat/api/src/test/java/org/apache/kafka/copycat/data/TimeTest.java b/copycat/api/src/test/java/org/apache/kafka/copycat/data/TimeTest.java new file mode 100644 index 0000000000000..8e54cb2538272 --- /dev/null +++ b/copycat/api/src/test/java/org/apache/kafka/copycat/data/TimeTest.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.copycat.data; + +import org.apache.kafka.copycat.errors.DataException; +import org.junit.Test; + +import java.util.Calendar; +import java.util.GregorianCalendar; +import java.util.TimeZone; + +import static org.junit.Assert.assertEquals; + +public class TimeTest { + private static final GregorianCalendar EPOCH; + private static final GregorianCalendar EPOCH_PLUS_DATE_COMPONENT; + private static final GregorianCalendar EPOCH_PLUS_TEN_THOUSAND_MILLIS; + static { + EPOCH = new GregorianCalendar(1970, Calendar.JANUARY, 1, 0, 0, 0); + EPOCH.setTimeZone(TimeZone.getTimeZone("UTC")); + + EPOCH_PLUS_TEN_THOUSAND_MILLIS = new GregorianCalendar(1970, Calendar.JANUARY, 1, 0, 0, 0); + EPOCH_PLUS_TEN_THOUSAND_MILLIS.setTimeZone(TimeZone.getTimeZone("UTC")); + EPOCH_PLUS_TEN_THOUSAND_MILLIS.add(Calendar.MILLISECOND, 10000); + + + EPOCH_PLUS_DATE_COMPONENT = new GregorianCalendar(1970, Calendar.JANUARY, 1, 0, 0, 0); + EPOCH_PLUS_DATE_COMPONENT.setTimeZone(TimeZone.getTimeZone("UTC")); + EPOCH_PLUS_DATE_COMPONENT.add(Calendar.DATE, 10000); + } + + @Test + public void testBuilder() { + Schema plain = Time.SCHEMA; + assertEquals(Time.LOGICAL_NAME, plain.name()); + assertEquals(1, (Object) plain.version()); + } + + @Test + public void testFromLogical() { + assertEquals(0, Time.fromLogical(Time.SCHEMA, EPOCH.getTime())); + assertEquals(10000, Time.fromLogical(Time.SCHEMA, EPOCH_PLUS_TEN_THOUSAND_MILLIS.getTime())); + } + + @Test(expected = DataException.class) + public void testFromLogicalInvalidSchema() { + Time.fromLogical(Time.builder().name("invalid").build(), EPOCH.getTime()); + } + + @Test(expected = DataException.class) + public void testFromLogicalInvalidHasDateComponents() { + Time.fromLogical(Time.SCHEMA, EPOCH_PLUS_DATE_COMPONENT.getTime()); + } + + @Test + public void testToLogical() { + assertEquals(EPOCH.getTime(), Time.toLogical(Time.SCHEMA, 0)); + assertEquals(EPOCH_PLUS_TEN_THOUSAND_MILLIS.getTime(), Time.toLogical(Time.SCHEMA, 10000)); + } + + @Test(expected = DataException.class) + public void testToLogicalInvalidSchema() { + Time.toLogical(Time.builder().name("invalid").build(), 0); + } +} diff --git a/copycat/api/src/test/java/org/apache/kafka/copycat/data/TimestampTest.java b/copycat/api/src/test/java/org/apache/kafka/copycat/data/TimestampTest.java new file mode 100644 index 0000000000000..cb5454cb4d02a --- /dev/null +++ b/copycat/api/src/test/java/org/apache/kafka/copycat/data/TimestampTest.java @@ -0,0 +1,75 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.data; + +import org.apache.kafka.copycat.errors.DataException; +import org.junit.Test; + +import java.util.Calendar; +import java.util.GregorianCalendar; +import java.util.TimeZone; + +import static org.junit.Assert.assertEquals; + +public class TimestampTest { + private static final GregorianCalendar EPOCH; + private static final GregorianCalendar EPOCH_PLUS_MILLIS; + + private static final int NUM_MILLIS = 2000000000; + private static final long TOTAL_MILLIS = ((long) NUM_MILLIS) * 2; + + static { + EPOCH = new GregorianCalendar(1970, Calendar.JANUARY, 1, 0, 0, 0); + EPOCH.setTimeZone(TimeZone.getTimeZone("UTC")); + + + EPOCH_PLUS_MILLIS = new GregorianCalendar(1970, Calendar.JANUARY, 1, 0, 0, 0); + EPOCH_PLUS_MILLIS.setTimeZone(TimeZone.getTimeZone("UTC")); + EPOCH_PLUS_MILLIS.add(Calendar.MILLISECOND, NUM_MILLIS); + EPOCH_PLUS_MILLIS.add(Calendar.MILLISECOND, NUM_MILLIS); + } + + @Test + public void testBuilder() { + Schema plain = Date.SCHEMA; + assertEquals(Date.LOGICAL_NAME, plain.name()); + assertEquals(1, (Object) plain.version()); + } + + @Test + public void testFromLogical() { + assertEquals(0L, Timestamp.fromLogical(Timestamp.SCHEMA, EPOCH.getTime())); + assertEquals(TOTAL_MILLIS, Timestamp.fromLogical(Timestamp.SCHEMA, EPOCH_PLUS_MILLIS.getTime())); + } + + @Test(expected = DataException.class) + public void testFromLogicalInvalidSchema() { + Timestamp.fromLogical(Timestamp.builder().name("invalid").build(), EPOCH.getTime()); + } + + @Test + public void testToLogical() { + assertEquals(EPOCH.getTime(), Timestamp.toLogical(Timestamp.SCHEMA, 0L)); + assertEquals(EPOCH_PLUS_MILLIS.getTime(), Timestamp.toLogical(Timestamp.SCHEMA, TOTAL_MILLIS)); + } + + @Test(expected = DataException.class) + public void testToLogicalInvalidSchema() { + Date.toLogical(Date.builder().name("invalid").build(), 0); + } +} diff --git a/copycat/api/src/test/java/org/apache/kafka/copycat/storage/StringConverterTest.java b/copycat/api/src/test/java/org/apache/kafka/copycat/storage/StringConverterTest.java new file mode 100644 index 0000000000000..3ea69c1923edc --- /dev/null +++ b/copycat/api/src/test/java/org/apache/kafka/copycat/storage/StringConverterTest.java @@ -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 org.apache.kafka.copycat.storage; + +import org.apache.kafka.copycat.data.Schema; +import org.apache.kafka.copycat.data.SchemaAndValue; +import org.junit.Test; + +import java.io.UnsupportedEncodingException; +import java.util.Collections; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertArrayEquals; + +public class StringConverterTest { + private static final String TOPIC = "topic"; + private static final String SAMPLE_STRING = "a string"; + + private StringConverter converter = new StringConverter(); + + @Test + public void testStringToBytes() throws UnsupportedEncodingException { + assertArrayEquals(SAMPLE_STRING.getBytes("UTF8"), converter.fromCopycatData(TOPIC, Schema.STRING_SCHEMA, SAMPLE_STRING)); + } + + @Test + public void testNonStringToBytes() throws UnsupportedEncodingException { + assertArrayEquals("true".getBytes("UTF8"), converter.fromCopycatData(TOPIC, Schema.BOOLEAN_SCHEMA, true)); + } + + @Test + public void testNullToBytes() { + assertEquals(null, converter.fromCopycatData(TOPIC, Schema.OPTIONAL_STRING_SCHEMA, null)); + } + + @Test + public void testToBytesIgnoresSchema() throws UnsupportedEncodingException { + assertArrayEquals("true".getBytes("UTF8"), converter.fromCopycatData(TOPIC, null, true)); + } + + @Test + public void testToBytesNonUtf8Encoding() throws UnsupportedEncodingException { + converter.configure(Collections.singletonMap("converter.encoding", "UTF-16"), true); + assertArrayEquals(SAMPLE_STRING.getBytes("UTF-16"), converter.fromCopycatData(TOPIC, Schema.STRING_SCHEMA, SAMPLE_STRING)); + } + + @Test + public void testBytesToString() { + SchemaAndValue data = converter.toCopycatData(TOPIC, SAMPLE_STRING.getBytes()); + assertEquals(Schema.OPTIONAL_STRING_SCHEMA, data.schema()); + assertEquals(SAMPLE_STRING, data.value()); + } + + @Test + public void testBytesNullToString() { + SchemaAndValue data = converter.toCopycatData(TOPIC, null); + assertEquals(Schema.OPTIONAL_STRING_SCHEMA, data.schema()); + assertEquals(null, data.value()); + } + + @Test + public void testBytesToStringNonUtf8Encoding() throws UnsupportedEncodingException { + converter.configure(Collections.singletonMap("converter.encoding", "UTF-16"), true); + SchemaAndValue data = converter.toCopycatData(TOPIC, SAMPLE_STRING.getBytes("UTF-16")); + assertEquals(Schema.OPTIONAL_STRING_SCHEMA, data.schema()); + assertEquals(SAMPLE_STRING, data.value()); + } +} diff --git a/copycat/api/src/test/java/org/apache/kafka/copycat/util/ConnectorUtilsTest.java b/copycat/api/src/test/java/org/apache/kafka/copycat/util/ConnectorUtilsTest.java new file mode 100644 index 0000000000000..e46967b642659 --- /dev/null +++ b/copycat/api/src/test/java/org/apache/kafka/copycat/util/ConnectorUtilsTest.java @@ -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 org.apache.kafka.copycat.util; + +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.junit.Assert.assertEquals; + +public class ConnectorUtilsTest { + + private static final List FIVE_ELEMENTS = Arrays.asList(1, 2, 3, 4, 5); + + @Test + public void testGroupPartitions() { + + List> grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 1); + assertEquals(Arrays.asList(FIVE_ELEMENTS), grouped); + + grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 2); + assertEquals(Arrays.asList(Arrays.asList(1, 2, 3), Arrays.asList(4, 5)), grouped); + + grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 3); + assertEquals(Arrays.asList(Arrays.asList(1, 2), + Arrays.asList(3, 4), + Arrays.asList(5)), grouped); + + grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 5); + assertEquals(Arrays.asList(Arrays.asList(1), + Arrays.asList(2), + Arrays.asList(3), + Arrays.asList(4), + Arrays.asList(5)), grouped); + + grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 7); + assertEquals(Arrays.asList(Arrays.asList(1), + Arrays.asList(2), + Arrays.asList(3), + Arrays.asList(4), + Arrays.asList(5), + Collections.EMPTY_LIST, + Collections.EMPTY_LIST), grouped); + } + + @Test(expected = IllegalArgumentException.class) + public void testGroupPartitionsInvalidCount() { + ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 0); + } +} diff --git a/copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSinkConnector.java b/copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSinkConnector.java new file mode 100644 index 0000000000000..6e2b04ddcdc98 --- /dev/null +++ b/copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSinkConnector.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.copycat.file; + +import org.apache.kafka.copycat.connector.Task; +import org.apache.kafka.copycat.sink.SinkConnector; + +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; + +/** + * Very simple connector that works with the console. This connector supports both source and + * sink modes via its 'mode' setting. + */ +public class FileStreamSinkConnector extends SinkConnector { + public static final String FILE_CONFIG = "file"; + + private String filename; + + @Override + public void start(Properties props) { + filename = props.getProperty(FILE_CONFIG); + } + + @Override + public Class taskClass() { + return FileStreamSinkTask.class; + } + + @Override + public List taskConfigs(int maxTasks) { + ArrayList configs = new ArrayList<>(); + for (int i = 0; i < maxTasks; i++) { + Properties config = new Properties(); + if (filename != null) + config.setProperty(FILE_CONFIG, filename); + configs.add(config); + } + return configs; + } + + @Override + public void stop() { + // Nothing to do since FileStreamSinkConnector has no background monitoring. + } +} diff --git a/copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSinkTask.java b/copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSinkTask.java new file mode 100644 index 0000000000000..9ea459c728013 --- /dev/null +++ b/copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSinkTask.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.copycat.file; + +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.copycat.errors.CopycatException; +import org.apache.kafka.copycat.sink.SinkRecord; +import org.apache.kafka.copycat.sink.SinkTask; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.PrintStream; +import java.util.Collection; +import java.util.Map; +import java.util.Properties; + +/** + * FileStreamSinkTask writes records to stdout or a file. + */ +public class FileStreamSinkTask extends SinkTask { + private static final Logger log = LoggerFactory.getLogger(FileStreamSinkTask.class); + + private PrintStream outputStream; + + public FileStreamSinkTask() { + } + + // for testing + public FileStreamSinkTask(PrintStream outputStream) { + this.outputStream = outputStream; + } + + @Override + public void start(Properties props) { + String filename = props.getProperty(FileStreamSinkConnector.FILE_CONFIG); + if (filename == null) { + outputStream = System.out; + } else { + try { + outputStream = new PrintStream(new FileOutputStream(filename, true)); + } catch (FileNotFoundException e) { + throw new CopycatException("Couldn't find or create file for FileStreamSinkTask", e); + } + } + } + + @Override + public void put(Collection sinkRecords) { + for (SinkRecord record : sinkRecords) { + outputStream.println(record.value()); + } + } + + @Override + public void flush(Map offsets) { + outputStream.flush(); + } + + @Override + public void stop() { + } +} diff --git a/copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSourceConnector.java b/copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSourceConnector.java new file mode 100644 index 0000000000000..716322f4b77c7 --- /dev/null +++ b/copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSourceConnector.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.copycat.file; + +import org.apache.kafka.copycat.connector.Task; +import org.apache.kafka.copycat.errors.CopycatException; +import org.apache.kafka.copycat.source.SourceConnector; + +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; + +/** + * Very simple connector that works with the console. This connector supports both source and + * sink modes via its 'mode' setting. + */ +public class FileStreamSourceConnector extends SourceConnector { + public static final String TOPIC_CONFIG = "topic"; + public static final String FILE_CONFIG = "file"; + + private String filename; + private String topic; + + @Override + public void start(Properties props) { + filename = props.getProperty(FILE_CONFIG); + topic = props.getProperty(TOPIC_CONFIG); + if (topic == null || topic.isEmpty()) + throw new CopycatException("FileStreamSourceConnector configuration must include 'topic' setting"); + if (topic.contains(",")) + throw new CopycatException("FileStreamSourceConnector should only have a single topic when used as a source."); + } + + @Override + public Class taskClass() { + return FileStreamSourceTask.class; + } + + @Override + public List taskConfigs(int maxTasks) { + ArrayList configs = new ArrayList<>(); + // Only one input stream makes sense. + Properties config = new Properties(); + if (filename != null) + config.setProperty(FILE_CONFIG, filename); + config.setProperty(TOPIC_CONFIG, topic); + configs.add(config); + return configs; + } + + @Override + public void stop() { + // Nothing to do since FileStreamSourceConnector has no background monitoring. + } +} diff --git a/copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSourceTask.java b/copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSourceTask.java new file mode 100644 index 0000000000000..cf71be3a00dff --- /dev/null +++ b/copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSourceTask.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.copycat.file; + +import org.apache.kafka.copycat.data.Schema; +import org.apache.kafka.copycat.errors.CopycatException; +import org.apache.kafka.copycat.source.SourceRecord; +import org.apache.kafka.copycat.source.SourceTask; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.*; +import java.util.*; + +/** + * FileStreamSourceTask reads from stdin or a file. + */ +public class FileStreamSourceTask extends SourceTask { + private static final Logger log = LoggerFactory.getLogger(FileStreamSourceTask.class); + public static final String FILENAME_FIELD = "filename"; + public static final String POSITION_FIELD = "position"; + private static final Schema VALUE_SCHEMA = Schema.STRING_SCHEMA; + + private String filename; + private InputStream stream; + private BufferedReader reader = null; + private char[] buffer = new char[1024]; + private int offset = 0; + private String topic = null; + + private Long streamOffset; + + @Override + public void start(Properties props) { + filename = props.getProperty(FileStreamSourceConnector.FILE_CONFIG); + if (filename == null || filename.isEmpty()) { + stream = System.in; + // Tracking offset for stdin doesn't make sense + streamOffset = null; + reader = new BufferedReader(new InputStreamReader(stream)); + } + topic = props.getProperty(FileStreamSourceConnector.TOPIC_CONFIG); + if (topic == null) + throw new CopycatException("ConsoleSourceTask config missing topic setting"); + } + + @Override + public List poll() throws InterruptedException { + if (stream == null) { + try { + stream = new FileInputStream(filename); + Map offset = context.offsetStorageReader().offset(Collections.singletonMap(FILENAME_FIELD, filename)); + if (offset != null) { + Object lastRecordedOffset = offset.get(POSITION_FIELD); + if (lastRecordedOffset != null && !(lastRecordedOffset instanceof Long)) + throw new CopycatException("Offset position is the incorrect type"); + if (lastRecordedOffset != null) { + log.debug("Found previous offset, trying to skip to file offset {}", lastRecordedOffset); + long skipLeft = (Long) lastRecordedOffset; + while (skipLeft > 0) { + try { + long skipped = stream.skip(skipLeft); + skipLeft -= skipped; + } catch (IOException e) { + log.error("Error while trying to seek to previous offset in file: ", e); + throw new CopycatException(e); + } + } + log.debug("Skipped to offset {}", lastRecordedOffset); + } + streamOffset = (lastRecordedOffset != null) ? (Long) lastRecordedOffset : 0L; + } else { + streamOffset = 0L; + } + reader = new BufferedReader(new InputStreamReader(stream)); + } catch (FileNotFoundException e) { + log.warn("Couldn't find file for FileStreamSourceTask, sleeping to wait for it to be created"); + synchronized (this) { + this.wait(1000); + } + return null; + } + } + + // Unfortunately we can't just use readLine() because it blocks in an uninterruptible way. + // Instead we have to manage splitting lines ourselves, using simple backoff when no new data + // is available. + try { + final BufferedReader readerCopy; + synchronized (this) { + readerCopy = reader; + } + if (readerCopy == null) + return null; + + ArrayList records = null; + + int nread = 0; + while (readerCopy.ready()) { + nread = readerCopy.read(buffer, offset, buffer.length - offset); + + if (nread > 0) { + offset += nread; + if (offset == buffer.length) { + char[] newbuf = new char[buffer.length * 2]; + System.arraycopy(buffer, 0, newbuf, 0, buffer.length); + buffer = newbuf; + } + + String line; + do { + line = extractLine(); + if (line != null) { + if (records == null) + records = new ArrayList<>(); + records.add(new SourceRecord(offsetKey(filename), offsetValue(streamOffset), topic, VALUE_SCHEMA, line)); + } + new ArrayList(); + } while (line != null); + } + } + + if (nread <= 0) + synchronized (this) { + this.wait(1000); + } + + return records; + } catch (IOException e) { + // Underlying stream was killed, probably as a result of calling stop. Allow to return + // null, and driving thread will handle any shutdown if necessary. + } + return null; + } + + private String extractLine() { + int until = -1, newStart = -1; + for (int i = 0; i < offset; i++) { + if (buffer[i] == '\n') { + until = i; + newStart = i + 1; + break; + } else if (buffer[i] == '\r') { + // We need to check for \r\n, so we must skip this if we can't check the next char + if (i + 1 >= offset) + return null; + + until = i; + newStart = (buffer[i + 1] == '\n') ? i + 2 : i + 1; + break; + } + } + + if (until != -1) { + String result = new String(buffer, 0, until); + System.arraycopy(buffer, newStart, buffer, 0, buffer.length - newStart); + offset = offset - newStart; + if (streamOffset != null) + streamOffset += newStart; + return result; + } else { + return null; + } + } + + @Override + public void stop() { + log.trace("Stopping"); + synchronized (this) { + try { + stream.close(); + log.trace("Closed input stream"); + } catch (IOException e) { + log.error("Failed to close ConsoleSourceTask stream: ", e); + } + this.notify(); + } + } + + private Map offsetKey(String filename) { + return Collections.singletonMap(FILENAME_FIELD, filename); + } + + private Map offsetValue(Long pos) { + return Collections.singletonMap(POSITION_FIELD, pos); + } +} diff --git a/copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSinkConnectorTest.java b/copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSinkConnectorTest.java new file mode 100644 index 0000000000000..ab5fd3b2863e4 --- /dev/null +++ b/copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSinkConnectorTest.java @@ -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 org.apache.kafka.copycat.file; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.copycat.connector.ConnectorContext; +import org.apache.kafka.copycat.sink.SinkConnector; +import org.junit.Before; +import org.junit.Test; +import org.powermock.api.easymock.PowerMock; + +import java.util.Arrays; +import java.util.List; +import java.util.Properties; + +import static org.junit.Assert.assertEquals; + +public class FileStreamSinkConnectorTest { + + private static final String MULTIPLE_TOPICS = "test1,test2"; + private static final String[] MULTIPLE_TOPICS_LIST + = MULTIPLE_TOPICS.split(","); + private static final List MULTIPLE_TOPICS_PARTITIONS = Arrays.asList( + new TopicPartition("test1", 1), new TopicPartition("test2", 2) + ); + private static final String FILENAME = "/afilename"; + + private FileStreamSinkConnector connector; + private ConnectorContext ctx; + private Properties sinkProperties; + + @Before + public void setup() { + connector = new FileStreamSinkConnector(); + ctx = PowerMock.createMock(ConnectorContext.class); + connector.initialize(ctx); + + sinkProperties = new Properties(); + sinkProperties.setProperty(SinkConnector.TOPICS_CONFIG, MULTIPLE_TOPICS); + sinkProperties.setProperty(FileStreamSinkConnector.FILE_CONFIG, FILENAME); + } + + @Test + public void testSinkTasks() { + PowerMock.replayAll(); + + connector.start(sinkProperties); + List taskConfigs = connector.taskConfigs(1); + assertEquals(1, taskConfigs.size()); + assertEquals(FILENAME, taskConfigs.get(0).getProperty(FileStreamSinkConnector.FILE_CONFIG)); + + taskConfigs = connector.taskConfigs(2); + assertEquals(2, taskConfigs.size()); + for (int i = 0; i < 2; i++) { + assertEquals(FILENAME, taskConfigs.get(0).getProperty(FileStreamSinkConnector.FILE_CONFIG)); + } + + PowerMock.verifyAll(); + } + + @Test + public void testTaskClass() { + PowerMock.replayAll(); + + connector.start(sinkProperties); + assertEquals(FileStreamSinkTask.class, connector.taskClass()); + + PowerMock.verifyAll(); + } +} diff --git a/copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSinkTaskTest.java b/copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSinkTaskTest.java new file mode 100644 index 0000000000000..ac8b5f19d2018 --- /dev/null +++ b/copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSinkTaskTest.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.copycat.file; + +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.copycat.data.Schema; +import org.apache.kafka.copycat.sink.SinkRecord; +import org.junit.Before; +import org.junit.Test; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.Arrays; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; + +public class FileStreamSinkTaskTest { + + private FileStreamSinkTask task; + private ByteArrayOutputStream os; + private PrintStream printStream; + + @Before + public void setup() { + os = new ByteArrayOutputStream(); + printStream = new PrintStream(os); + task = new FileStreamSinkTask(printStream); + } + + @Test + public void testPutFlush() { + HashMap offsets = new HashMap<>(); + + // We do not call task.start() since it would override the output stream + + task.put(Arrays.asList( + new SinkRecord("topic1", 0, null, null, Schema.STRING_SCHEMA, "line1", 1) + )); + offsets.put(new TopicPartition("topic1", 0), new OffsetAndMetadata(1L)); + task.flush(offsets); + assertEquals("line1\n", os.toString()); + + task.put(Arrays.asList( + new SinkRecord("topic1", 0, null, null, Schema.STRING_SCHEMA, "line2", 2), + new SinkRecord("topic2", 0, null, null, Schema.STRING_SCHEMA, "line3", 1) + )); + offsets.put(new TopicPartition("topic1", 0), new OffsetAndMetadata(2L)); + offsets.put(new TopicPartition("topic2", 0), new OffsetAndMetadata(1L)); + task.flush(offsets); + assertEquals("line1\nline2\nline3\n", os.toString()); + } +} diff --git a/copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSourceConnectorTest.java b/copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSourceConnectorTest.java new file mode 100644 index 0000000000000..41e15a08332d4 --- /dev/null +++ b/copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSourceConnectorTest.java @@ -0,0 +1,104 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.file; + +import org.apache.kafka.copycat.connector.ConnectorContext; +import org.apache.kafka.copycat.errors.CopycatException; +import org.junit.Before; +import org.junit.Test; +import org.powermock.api.easymock.PowerMock; + +import java.util.List; +import java.util.Properties; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +public class FileStreamSourceConnectorTest { + + private static final String SINGLE_TOPIC = "test"; + private static final String MULTIPLE_TOPICS = "test1,test2"; + private static final String FILENAME = "/somefilename"; + + private FileStreamSourceConnector connector; + private ConnectorContext ctx; + private Properties sourceProperties; + + @Before + public void setup() { + connector = new FileStreamSourceConnector(); + ctx = PowerMock.createMock(ConnectorContext.class); + connector.initialize(ctx); + + sourceProperties = new Properties(); + sourceProperties.setProperty(FileStreamSourceConnector.TOPIC_CONFIG, SINGLE_TOPIC); + sourceProperties.setProperty(FileStreamSourceConnector.FILE_CONFIG, FILENAME); + } + + @Test + public void testSourceTasks() { + PowerMock.replayAll(); + + connector.start(sourceProperties); + List taskConfigs = connector.taskConfigs(1); + assertEquals(1, taskConfigs.size()); + assertEquals(FILENAME, + taskConfigs.get(0).getProperty(FileStreamSourceConnector.FILE_CONFIG)); + assertEquals(SINGLE_TOPIC, + taskConfigs.get(0).getProperty(FileStreamSourceConnector.TOPIC_CONFIG)); + + // Should be able to return fewer than requested # + taskConfigs = connector.taskConfigs(2); + assertEquals(1, taskConfigs.size()); + assertEquals(FILENAME, + taskConfigs.get(0).getProperty(FileStreamSourceConnector.FILE_CONFIG)); + assertEquals(SINGLE_TOPIC, + taskConfigs.get(0).getProperty(FileStreamSourceConnector.TOPIC_CONFIG)); + + PowerMock.verifyAll(); + } + + @Test + public void testSourceTasksStdin() { + PowerMock.replayAll(); + + sourceProperties.remove(FileStreamSourceConnector.FILE_CONFIG); + connector.start(sourceProperties); + List taskConfigs = connector.taskConfigs(1); + assertEquals(1, taskConfigs.size()); + assertNull(taskConfigs.get(0).getProperty(FileStreamSourceConnector.FILE_CONFIG)); + + PowerMock.verifyAll(); + } + + @Test(expected = CopycatException.class) + public void testMultipleSourcesInvalid() { + sourceProperties.setProperty(FileStreamSourceConnector.TOPIC_CONFIG, MULTIPLE_TOPICS); + connector.start(sourceProperties); + } + + @Test + public void testTaskClass() { + PowerMock.replayAll(); + + connector.start(sourceProperties); + assertEquals(FileStreamSourceTask.class, connector.taskClass()); + + PowerMock.verifyAll(); + } +} diff --git a/copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSourceTaskTest.java b/copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSourceTaskTest.java new file mode 100644 index 0000000000000..d2781c963bc9e --- /dev/null +++ b/copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSourceTaskTest.java @@ -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. + **/ + +package org.apache.kafka.copycat.file; + +import org.apache.kafka.copycat.errors.CopycatException; +import org.apache.kafka.copycat.source.SourceRecord; +import org.apache.kafka.copycat.source.SourceTaskContext; +import org.apache.kafka.copycat.storage.OffsetStorageReader; +import org.easymock.EasyMock; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.powermock.api.easymock.PowerMock; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static org.junit.Assert.assertEquals; + +public class FileStreamSourceTaskTest { + + private static final String TOPIC = "test"; + + private File tempFile; + private Properties config; + private OffsetStorageReader offsetStorageReader; + private FileStreamSourceTask task; + + private boolean verifyMocks = false; + + @Before + public void setup() throws IOException { + tempFile = File.createTempFile("file-stream-source-task-test", null); + config = new Properties(); + config.setProperty(FileStreamSourceConnector.FILE_CONFIG, tempFile.getAbsolutePath()); + config.setProperty(FileStreamSourceConnector.TOPIC_CONFIG, TOPIC); + task = new FileStreamSourceTask(); + offsetStorageReader = PowerMock.createMock(OffsetStorageReader.class); + task.initialize(new SourceTaskContext(offsetStorageReader)); + } + + @After + public void teardown() { + tempFile.delete(); + + if (verifyMocks) + PowerMock.verifyAll(); + } + + private void replay() { + PowerMock.replayAll(); + verifyMocks = true; + } + + @Test + public void testNormalLifecycle() throws InterruptedException, IOException { + expectOffsetLookupReturnNone(); + replay(); + + task.start(config); + + FileOutputStream os = new FileOutputStream(tempFile); + assertEquals(null, task.poll()); + os.write("partial line".getBytes()); + os.flush(); + assertEquals(null, task.poll()); + os.write(" finished\n".getBytes()); + os.flush(); + List records = task.poll(); + assertEquals(1, records.size()); + assertEquals(TOPIC, records.get(0).topic()); + assertEquals("partial line finished", records.get(0).value()); + assertEquals(Collections.singletonMap(FileStreamSourceTask.FILENAME_FIELD, tempFile.getAbsolutePath()), records.get(0).sourcePartition()); + assertEquals(Collections.singletonMap(FileStreamSourceTask.POSITION_FIELD, 22L), records.get(0).sourceOffset()); + assertEquals(null, task.poll()); + + // Different line endings, and make sure the final \r doesn't result in a line until we can + // read the subsequent byte. + os.write("line1\rline2\r\nline3\nline4\n\r".getBytes()); + os.flush(); + records = task.poll(); + assertEquals(4, records.size()); + assertEquals("line1", records.get(0).value()); + assertEquals(Collections.singletonMap(FileStreamSourceTask.FILENAME_FIELD, tempFile.getAbsolutePath()), records.get(0).sourcePartition()); + assertEquals(Collections.singletonMap(FileStreamSourceTask.POSITION_FIELD, 28L), records.get(0).sourceOffset()); + assertEquals("line2", records.get(1).value()); + assertEquals(Collections.singletonMap(FileStreamSourceTask.FILENAME_FIELD, tempFile.getAbsolutePath()), records.get(1).sourcePartition()); + assertEquals(Collections.singletonMap(FileStreamSourceTask.POSITION_FIELD, 35L), records.get(1).sourceOffset()); + assertEquals("line3", records.get(2).value()); + assertEquals(Collections.singletonMap(FileStreamSourceTask.FILENAME_FIELD, tempFile.getAbsolutePath()), records.get(2).sourcePartition()); + assertEquals(Collections.singletonMap(FileStreamSourceTask.POSITION_FIELD, 41L), records.get(2).sourceOffset()); + assertEquals("line4", records.get(3).value()); + assertEquals(Collections.singletonMap(FileStreamSourceTask.FILENAME_FIELD, tempFile.getAbsolutePath()), records.get(3).sourcePartition()); + assertEquals(Collections.singletonMap(FileStreamSourceTask.POSITION_FIELD, 47L), records.get(3).sourceOffset()); + + os.write("subsequent text".getBytes()); + os.flush(); + records = task.poll(); + assertEquals(1, records.size()); + assertEquals("", records.get(0).value()); + assertEquals(Collections.singletonMap(FileStreamSourceTask.FILENAME_FIELD, tempFile.getAbsolutePath()), records.get(0).sourcePartition()); + assertEquals(Collections.singletonMap(FileStreamSourceTask.POSITION_FIELD, 48L), records.get(0).sourceOffset()); + + task.stop(); + } + + @Test(expected = CopycatException.class) + public void testMissingTopic() throws InterruptedException { + replay(); + + config.remove(FileStreamSourceConnector.TOPIC_CONFIG); + task.start(config); + } + + public void testInvalidFile() throws InterruptedException { + config.setProperty(FileStreamSourceConnector.FILE_CONFIG, "bogusfilename"); + task.start(config); + // Currently the task retries indefinitely if the file isn't found, but shouldn't return any data. + for (int i = 0; i < 100; i++) + assertEquals(null, task.poll()); + } + + + private void expectOffsetLookupReturnNone() { + EasyMock.expect(offsetStorageReader.offset(EasyMock.anyObject(Map.class))).andReturn(null); + } +} \ No newline at end of file diff --git a/copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonConverter.java b/copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonConverter.java new file mode 100644 index 0000000000000..ca8f029fcba6c --- /dev/null +++ b/copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonConverter.java @@ -0,0 +1,735 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

            + * http://www.apache.org/licenses/LICENSE-2.0 + *

            + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.json; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.kafka.common.cache.Cache; +import org.apache.kafka.common.cache.LRUCache; +import org.apache.kafka.common.cache.SynchronizedCache; +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.copycat.data.*; +import org.apache.kafka.copycat.errors.DataException; +import org.apache.kafka.copycat.storage.Converter; + +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +/** + * Implementation of Converter that uses JSON to store schemas and objects. + */ +public class JsonConverter implements Converter { + private static final String SCHEMAS_ENABLE_CONFIG = "schemas.enable"; + private static final boolean SCHEMAS_ENABLE_DEFAULT = true; + private static final String SCHEMAS_CACHE_CONFIG = "schemas.cache.size"; + private static final int SCHEMAS_CACHE_SIZE_DEFAULT = 1000; + + private static final HashMap TO_COPYCAT_CONVERTERS = new HashMap<>(); + + private static Object checkOptionalAndDefault(Schema schema) { + if (schema.defaultValue() != null) + return schema.defaultValue(); + if (schema.isOptional()) + return null; + throw new DataException("Invalid null value for required field"); + } + + static { + TO_COPYCAT_CONVERTERS.put(Schema.Type.BOOLEAN, new JsonToCopycatTypeConverter() { + @Override + public Object convert(Schema schema, JsonNode value) { + if (value.isNull()) return checkOptionalAndDefault(schema); + return value.booleanValue(); + } + }); + TO_COPYCAT_CONVERTERS.put(Schema.Type.INT8, new JsonToCopycatTypeConverter() { + @Override + public Object convert(Schema schema, JsonNode value) { + if (value.isNull()) return checkOptionalAndDefault(schema); + return (byte) value.intValue(); + } + }); + TO_COPYCAT_CONVERTERS.put(Schema.Type.INT16, new JsonToCopycatTypeConverter() { + @Override + public Object convert(Schema schema, JsonNode value) { + if (value.isNull()) return checkOptionalAndDefault(schema); + return (short) value.intValue(); + } + }); + TO_COPYCAT_CONVERTERS.put(Schema.Type.INT32, new JsonToCopycatTypeConverter() { + @Override + public Object convert(Schema schema, JsonNode value) { + if (value.isNull()) return checkOptionalAndDefault(schema); + return value.intValue(); + } + }); + TO_COPYCAT_CONVERTERS.put(Schema.Type.INT64, new JsonToCopycatTypeConverter() { + @Override + public Object convert(Schema schema, JsonNode value) { + if (value.isNull()) return checkOptionalAndDefault(schema); + return value.longValue(); + } + }); + TO_COPYCAT_CONVERTERS.put(Schema.Type.FLOAT32, new JsonToCopycatTypeConverter() { + @Override + public Object convert(Schema schema, JsonNode value) { + if (value.isNull()) return checkOptionalAndDefault(schema); + return value.floatValue(); + } + }); + TO_COPYCAT_CONVERTERS.put(Schema.Type.FLOAT64, new JsonToCopycatTypeConverter() { + @Override + public Object convert(Schema schema, JsonNode value) { + if (value.isNull()) return checkOptionalAndDefault(schema); + return value.doubleValue(); + } + }); + TO_COPYCAT_CONVERTERS.put(Schema.Type.BYTES, new JsonToCopycatTypeConverter() { + @Override + public Object convert(Schema schema, JsonNode value) { + try { + if (value.isNull()) return checkOptionalAndDefault(schema); + return value.binaryValue(); + } catch (IOException e) { + throw new DataException("Invalid bytes field", e); + } + } + }); + TO_COPYCAT_CONVERTERS.put(Schema.Type.STRING, new JsonToCopycatTypeConverter() { + @Override + public Object convert(Schema schema, JsonNode value) { + if (value.isNull()) return checkOptionalAndDefault(schema); + return value.textValue(); + } + }); + TO_COPYCAT_CONVERTERS.put(Schema.Type.ARRAY, new JsonToCopycatTypeConverter() { + @Override + public Object convert(Schema schema, JsonNode value) { + if (value.isNull()) return checkOptionalAndDefault(schema); + + Schema elemSchema = schema == null ? null : schema.valueSchema(); + ArrayList result = new ArrayList<>(); + for (JsonNode elem : value) { + result.add(convertToCopycat(elemSchema, elem)); + } + return result; + } + }); + TO_COPYCAT_CONVERTERS.put(Schema.Type.MAP, new JsonToCopycatTypeConverter() { + @Override + public Object convert(Schema schema, JsonNode value) { + if (value.isNull()) return checkOptionalAndDefault(schema); + + Schema keySchema = schema == null ? null : schema.keySchema(); + Schema valueSchema = schema == null ? null : schema.valueSchema(); + + // If the map uses strings for keys, it should be encoded in the natural JSON format. If it uses other + // primitive types or a complex type as a key, it will be encoded as a list of pairs. If we don't have a + // schema, we default to encoding in a Map. + Map result = new HashMap<>(); + if (schema == null || keySchema.type() == Schema.Type.STRING) { + if (!value.isObject()) + throw new DataException("Map's with string fields should be encoded as JSON objects, but found " + value.getNodeType()); + Iterator> fieldIt = value.fields(); + while (fieldIt.hasNext()) { + Map.Entry entry = fieldIt.next(); + result.put(entry.getKey(), convertToCopycat(valueSchema, entry.getValue())); + } + } else { + if (!value.isArray()) + throw new DataException("Map's with non-string fields should be encoded as JSON array of tuples, but found " + value.getNodeType()); + for (JsonNode entry : value) { + if (!entry.isArray()) + throw new DataException("Found invalid map entry instead of array tuple: " + entry.getNodeType()); + if (entry.size() != 2) + throw new DataException("Found invalid map entry, expected length 2 but found :" + entry.size()); + result.put(convertToCopycat(keySchema, entry.get(0)), + convertToCopycat(valueSchema, entry.get(1))); + } + } + return result; + } + }); + TO_COPYCAT_CONVERTERS.put(Schema.Type.STRUCT, new JsonToCopycatTypeConverter() { + @Override + public Object convert(Schema schema, JsonNode value) { + if (value.isNull()) return checkOptionalAndDefault(schema); + + if (!value.isObject()) + throw new DataException("Structs should be encoded as JSON objects, but found " + value.getNodeType()); + + // We only have ISchema here but need Schema, so we need to materialize the actual schema. Using ISchema + // avoids having to materialize the schema for non-Struct types but it cannot be avoided for Structs since + // they require a schema to be provided at construction. However, the schema is only a SchemaBuilder during + // translation of schemas to JSON; during the more common translation of data to JSON, the call to schema.schema() + // just returns the schema Object and has no overhead. + Struct result = new Struct(schema.schema()); + for (Field field : schema.fields()) + result.put(field, convertToCopycat(field.schema(), value.get(field.name()))); + + return result; + } + }); + } + + // Convert values in Copycat form into their logical types. These logical converters are discovered by logical type + // names specified in the field + private static final HashMap TO_COPYCAT_LOGICAL_CONVERTERS = new HashMap<>(); + static { + TO_COPYCAT_LOGICAL_CONVERTERS.put(Decimal.LOGICAL_NAME, new LogicalTypeConverter() { + @Override + public Object convert(Schema schema, Object value) { + if (!(value instanceof byte[])) + throw new DataException("Invalid type for Decimal, underlying representation should be bytes but was " + value.getClass()); + return Decimal.toLogical(schema, (byte[]) value); + } + }); + + TO_COPYCAT_LOGICAL_CONVERTERS.put(Date.LOGICAL_NAME, new LogicalTypeConverter() { + @Override + public Object convert(Schema schema, Object value) { + if (!(value instanceof Integer)) + throw new DataException("Invalid type for Date, underlying representation should be int32 but was " + value.getClass()); + return Date.toLogical(schema, (int) value); + } + }); + + TO_COPYCAT_LOGICAL_CONVERTERS.put(Time.LOGICAL_NAME, new LogicalTypeConverter() { + @Override + public Object convert(Schema schema, Object value) { + if (!(value instanceof Integer)) + throw new DataException("Invalid type for Time, underlying representation should be int32 but was " + value.getClass()); + return Time.toLogical(schema, (int) value); + } + }); + + TO_COPYCAT_LOGICAL_CONVERTERS.put(Timestamp.LOGICAL_NAME, new LogicalTypeConverter() { + @Override + public Object convert(Schema schema, Object value) { + if (!(value instanceof Long)) + throw new DataException("Invalid type for Timestamp, underlying representation should be int64 but was " + value.getClass()); + return Timestamp.toLogical(schema, (long) value); + } + }); + } + + private static final HashMap TO_JSON_LOGICAL_CONVERTERS = new HashMap<>(); + static { + TO_JSON_LOGICAL_CONVERTERS.put(Decimal.LOGICAL_NAME, new LogicalTypeConverter() { + @Override + public Object convert(Schema schema, Object value) { + if (!(value instanceof BigDecimal)) + throw new DataException("Invalid type for Decimal, expected BigDecimal but was " + value.getClass()); + return Decimal.fromLogical(schema, (BigDecimal) value); + } + }); + + TO_JSON_LOGICAL_CONVERTERS.put(Date.LOGICAL_NAME, new LogicalTypeConverter() { + @Override + public Object convert(Schema schema, Object value) { + if (!(value instanceof java.util.Date)) + throw new DataException("Invalid type for Date, expected Date but was " + value.getClass()); + return Date.fromLogical(schema, (java.util.Date) value); + } + }); + + TO_JSON_LOGICAL_CONVERTERS.put(Time.LOGICAL_NAME, new LogicalTypeConverter() { + @Override + public Object convert(Schema schema, Object value) { + if (!(value instanceof java.util.Date)) + throw new DataException("Invalid type for Time, expected Date but was " + value.getClass()); + return Time.fromLogical(schema, (java.util.Date) value); + } + }); + + TO_JSON_LOGICAL_CONVERTERS.put(Timestamp.LOGICAL_NAME, new LogicalTypeConverter() { + @Override + public Object convert(Schema schema, Object value) { + if (!(value instanceof java.util.Date)) + throw new DataException("Invalid type for Timestamp, expected Date but was " + value.getClass()); + return Timestamp.fromLogical(schema, (java.util.Date) value); + } + }); + } + + + private boolean enableSchemas = SCHEMAS_ENABLE_DEFAULT; + private int cacheSize = SCHEMAS_CACHE_SIZE_DEFAULT; + private Cache fromCopycatSchemaCache; + private Cache toCopycatSchemaCache; + + private final JsonSerializer serializer = new JsonSerializer(); + private final JsonDeserializer deserializer = new JsonDeserializer(); + + @Override + public void configure(Map configs, boolean isKey) { + Object enableConfigsVal = configs.get(SCHEMAS_ENABLE_CONFIG); + if (enableConfigsVal != null) + enableSchemas = enableConfigsVal.toString().equals("true"); + + serializer.configure(configs, isKey); + deserializer.configure(configs, isKey); + + Object cacheSizeVal = configs.get(SCHEMAS_CACHE_SIZE_DEFAULT); + if (cacheSizeVal != null) + cacheSize = (int) cacheSizeVal; + fromCopycatSchemaCache = new SynchronizedCache<>(new LRUCache(cacheSize)); + toCopycatSchemaCache = new SynchronizedCache<>(new LRUCache(cacheSize)); + } + + @Override + public byte[] fromCopycatData(String topic, Schema schema, Object value) { + JsonNode jsonValue = enableSchemas ? convertToJsonWithEnvelope(schema, value) : convertToJsonWithoutEnvelope(schema, value); + try { + return serializer.serialize(topic, jsonValue); + } catch (SerializationException e) { + throw new DataException("Converting Copycat data to byte[] failed due to serialization error: ", e); + } + } + + @Override + public SchemaAndValue toCopycatData(String topic, byte[] value) { + JsonNode jsonValue; + try { + jsonValue = deserializer.deserialize(topic, value); + } catch (SerializationException e) { + throw new DataException("Converting byte[] to Copycat data failed due to serialization error: ", e); + } + + if (enableSchemas && (jsonValue == null || !jsonValue.isObject() || jsonValue.size() != 2 || !jsonValue.has("schema") || !jsonValue.has("payload"))) + throw new DataException("JsonDeserializer with schemas.enable requires \"schema\" and \"payload\" fields and may not contain additional fields"); + + // The deserialized data should either be an envelope object containing the schema and the payload or the schema + // was stripped during serialization and we need to fill in an all-encompassing schema. + if (!enableSchemas) { + ObjectNode envelope = JsonNodeFactory.instance.objectNode(); + envelope.set("schema", null); + envelope.set("payload", jsonValue); + jsonValue = envelope; + } + + return jsonToCopycat(jsonValue); + } + + private SchemaAndValue jsonToCopycat(JsonNode jsonValue) { + if (jsonValue == null) + return SchemaAndValue.NULL; + + if (!jsonValue.isObject() || jsonValue.size() != 2 || !jsonValue.has(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME) || !jsonValue.has(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME)) + throw new DataException("JSON value converted to Copycat must be in envelope containing schema"); + + Schema schema = asCopycatSchema(jsonValue.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + return new SchemaAndValue(schema, convertToCopycat(schema, jsonValue.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME))); + } + + private ObjectNode asJsonSchema(Schema schema) { + if (schema == null) + return null; + + ObjectNode cached = fromCopycatSchemaCache.get(schema); + if (cached != null) + return cached; + + final ObjectNode jsonSchema; + switch (schema.type()) { + case BOOLEAN: + jsonSchema = JsonSchema.BOOLEAN_SCHEMA.deepCopy(); + break; + case BYTES: + jsonSchema = JsonSchema.BYTES_SCHEMA.deepCopy(); + break; + case FLOAT64: + jsonSchema = JsonSchema.DOUBLE_SCHEMA.deepCopy(); + break; + case FLOAT32: + jsonSchema = JsonSchema.FLOAT_SCHEMA.deepCopy(); + break; + case INT8: + jsonSchema = JsonSchema.INT8_SCHEMA.deepCopy(); + break; + case INT16: + jsonSchema = JsonSchema.INT16_SCHEMA.deepCopy(); + break; + case INT32: + jsonSchema = JsonSchema.INT32_SCHEMA.deepCopy(); + break; + case INT64: + jsonSchema = JsonSchema.INT64_SCHEMA.deepCopy(); + break; + case STRING: + jsonSchema = JsonSchema.STRING_SCHEMA.deepCopy(); + break; + case ARRAY: + jsonSchema = JsonNodeFactory.instance.objectNode().put(JsonSchema.SCHEMA_TYPE_FIELD_NAME, JsonSchema.ARRAY_TYPE_NAME); + jsonSchema.set(JsonSchema.ARRAY_ITEMS_FIELD_NAME, asJsonSchema(schema.valueSchema())); + break; + case MAP: + jsonSchema = JsonNodeFactory.instance.objectNode().put(JsonSchema.SCHEMA_TYPE_FIELD_NAME, JsonSchema.MAP_TYPE_NAME); + jsonSchema.set(JsonSchema.MAP_KEY_FIELD_NAME, asJsonSchema(schema.keySchema())); + jsonSchema.set(JsonSchema.MAP_VALUE_FIELD_NAME, asJsonSchema(schema.valueSchema())); + break; + case STRUCT: + jsonSchema = JsonNodeFactory.instance.objectNode().put(JsonSchema.SCHEMA_TYPE_FIELD_NAME, JsonSchema.STRUCT_TYPE_NAME); + ArrayNode fields = JsonNodeFactory.instance.arrayNode(); + for (Field field : schema.fields()) { + ObjectNode fieldJsonSchema = asJsonSchema(field.schema()); + fieldJsonSchema.put(JsonSchema.STRUCT_FIELD_NAME_FIELD_NAME, field.name()); + fields.add(fieldJsonSchema); + } + jsonSchema.set(JsonSchema.STRUCT_FIELDS_FIELD_NAME, fields); + break; + default: + throw new DataException("Couldn't translate unsupported schema type " + schema + "."); + } + + jsonSchema.put(JsonSchema.SCHEMA_OPTIONAL_FIELD_NAME, schema.isOptional()); + if (schema.name() != null) + jsonSchema.put(JsonSchema.SCHEMA_NAME_FIELD_NAME, schema.name()); + if (schema.version() != null) + jsonSchema.put(JsonSchema.SCHEMA_VERSION_FIELD_NAME, schema.version()); + if (schema.doc() != null) + jsonSchema.put(JsonSchema.SCHEMA_DOC_FIELD_NAME, schema.doc()); + if (schema.parameters() != null) { + ObjectNode jsonSchemaParams = JsonNodeFactory.instance.objectNode(); + for (Map.Entry prop : schema.parameters().entrySet()) + jsonSchemaParams.put(prop.getKey(), prop.getValue()); + jsonSchema.put(JsonSchema.SCHEMA_PARAMETERS_FIELD_NAME, jsonSchemaParams); + } + if (schema.defaultValue() != null) + jsonSchema.set(JsonSchema.SCHEMA_DEFAULT_FIELD_NAME, convertToJson(schema, schema.defaultValue())); + + fromCopycatSchemaCache.put(schema, jsonSchema); + return jsonSchema; + } + + + private Schema asCopycatSchema(JsonNode jsonSchema) { + if (jsonSchema.isNull()) + return null; + + Schema cached = toCopycatSchemaCache.get(jsonSchema); + if (cached != null) + return cached; + + JsonNode schemaTypeNode = jsonSchema.get(JsonSchema.SCHEMA_TYPE_FIELD_NAME); + if (schemaTypeNode == null || !schemaTypeNode.isTextual()) + throw new DataException("Schema must contain 'type' field"); + + final SchemaBuilder builder; + switch (schemaTypeNode.textValue()) { + case JsonSchema.BOOLEAN_TYPE_NAME: + builder = SchemaBuilder.bool(); + break; + case JsonSchema.INT8_TYPE_NAME: + builder = SchemaBuilder.int8(); + break; + case JsonSchema.INT16_TYPE_NAME: + builder = SchemaBuilder.int16(); + break; + case JsonSchema.INT32_TYPE_NAME: + builder = SchemaBuilder.int32(); + break; + case JsonSchema.INT64_TYPE_NAME: + builder = SchemaBuilder.int64(); + break; + case JsonSchema.FLOAT_TYPE_NAME: + builder = SchemaBuilder.float32(); + break; + case JsonSchema.DOUBLE_TYPE_NAME: + builder = SchemaBuilder.float64(); + break; + case JsonSchema.BYTES_TYPE_NAME: + builder = SchemaBuilder.bytes(); + break; + case JsonSchema.STRING_TYPE_NAME: + builder = SchemaBuilder.string(); + break; + case JsonSchema.ARRAY_TYPE_NAME: + JsonNode elemSchema = jsonSchema.get(JsonSchema.ARRAY_ITEMS_FIELD_NAME); + if (elemSchema == null) + throw new DataException("Array schema did not specify the element type"); + builder = SchemaBuilder.array(asCopycatSchema(elemSchema)); + break; + case JsonSchema.MAP_TYPE_NAME: + JsonNode keySchema = jsonSchema.get(JsonSchema.MAP_KEY_FIELD_NAME); + if (keySchema == null) + throw new DataException("Map schema did not specify the key type"); + JsonNode valueSchema = jsonSchema.get(JsonSchema.MAP_VALUE_FIELD_NAME); + if (valueSchema == null) + throw new DataException("Map schema did not specify the value type"); + builder = SchemaBuilder.map(asCopycatSchema(keySchema), asCopycatSchema(valueSchema)); + break; + case JsonSchema.STRUCT_TYPE_NAME: + builder = SchemaBuilder.struct(); + JsonNode fields = jsonSchema.get(JsonSchema.STRUCT_FIELDS_FIELD_NAME); + if (fields == null || !fields.isArray()) + throw new DataException("Struct schema's \"fields\" argument is not an array."); + for (JsonNode field : fields) { + JsonNode jsonFieldName = field.get(JsonSchema.STRUCT_FIELD_NAME_FIELD_NAME); + if (jsonFieldName == null || !jsonFieldName.isTextual()) + throw new DataException("Struct schema's field name not specified properly"); + builder.field(jsonFieldName.asText(), asCopycatSchema(field)); + } + break; + default: + throw new DataException("Unknown schema type: " + schemaTypeNode.textValue()); + } + + + JsonNode schemaOptionalNode = jsonSchema.get(JsonSchema.SCHEMA_OPTIONAL_FIELD_NAME); + if (schemaOptionalNode != null && schemaOptionalNode.isBoolean() && schemaOptionalNode.booleanValue()) + builder.optional(); + else + builder.required(); + + JsonNode schemaNameNode = jsonSchema.get(JsonSchema.SCHEMA_NAME_FIELD_NAME); + if (schemaNameNode != null && schemaNameNode.isTextual()) + builder.name(schemaNameNode.textValue()); + + JsonNode schemaVersionNode = jsonSchema.get(JsonSchema.SCHEMA_VERSION_FIELD_NAME); + if (schemaVersionNode != null && schemaVersionNode.isIntegralNumber()) { + builder.version(schemaVersionNode.intValue()); + } + + JsonNode schemaDocNode = jsonSchema.get(JsonSchema.SCHEMA_DOC_FIELD_NAME); + if (schemaDocNode != null && schemaDocNode.isTextual()) + builder.doc(schemaDocNode.textValue()); + + JsonNode schemaParamsNode = jsonSchema.get(JsonSchema.SCHEMA_PARAMETERS_FIELD_NAME); + if (schemaParamsNode != null && schemaParamsNode.isObject()) { + Iterator> paramsIt = schemaParamsNode.fields(); + while (paramsIt.hasNext()) { + Map.Entry entry = paramsIt.next(); + JsonNode paramValue = entry.getValue(); + if (!paramValue.isTextual()) + throw new DataException("Schema parameters must have string values."); + builder.parameter(entry.getKey(), paramValue.textValue()); + } + } + + JsonNode schemaDefaultNode = jsonSchema.get(JsonSchema.SCHEMA_DEFAULT_FIELD_NAME); + if (schemaDefaultNode != null) + builder.defaultValue(convertToCopycat(builder, schemaDefaultNode)); + + Schema result = builder.build(); + toCopycatSchemaCache.put(jsonSchema, result); + return result; + } + + + /** + * Convert this object, in org.apache.kafka.copycat.data format, into a JSON object with an envelope object + * containing schema and payload fields. + * @param schema the schema for the data + * @param value the value + * @return JsonNode-encoded version + */ + private JsonNode convertToJsonWithEnvelope(Schema schema, Object value) { + return new JsonSchema.Envelope(asJsonSchema(schema), convertToJson(schema, value)).toJsonNode(); + } + + private JsonNode convertToJsonWithoutEnvelope(Schema schema, Object value) { + return convertToJson(schema, value); + } + + /** + * Convert this object, in the org.apache.kafka.copycat.data format, into a JSON object, returning both the schema + * and the converted object. + */ + private static JsonNode convertToJson(Schema schema, Object logicalValue) { + if (logicalValue == null) { + if (schema == null) // Any schema is valid and we don't have a default, so treat this as an optional schema + return null; + if (schema.defaultValue() != null) + return convertToJson(schema, schema.defaultValue()); + if (schema.isOptional()) + return JsonNodeFactory.instance.nullNode(); + throw new DataException("Conversion error: null value for field that is required and has no default value"); + } + + Object value = logicalValue; + if (schema != null && schema.name() != null) { + LogicalTypeConverter logicalConverter = TO_JSON_LOGICAL_CONVERTERS.get(schema.name()); + if (logicalConverter != null) + value = logicalConverter.convert(schema, logicalValue); + } + + try { + final Schema.Type schemaType; + if (schema == null) { + schemaType = CopycatSchema.schemaType(value.getClass()); + if (schemaType == null) + throw new DataException("Java class " + value.getClass() + " does not have corresponding schema type."); + } else { + schemaType = schema.type(); + } + switch (schemaType) { + case INT8: + return JsonNodeFactory.instance.numberNode((Byte) value); + case INT16: + return JsonNodeFactory.instance.numberNode((Short) value); + case INT32: + return JsonNodeFactory.instance.numberNode((Integer) value); + case INT64: + return JsonNodeFactory.instance.numberNode((Long) value); + case FLOAT32: + return JsonNodeFactory.instance.numberNode((Float) value); + case FLOAT64: + return JsonNodeFactory.instance.numberNode((Double) value); + case BOOLEAN: + return JsonNodeFactory.instance.booleanNode((Boolean) value); + case STRING: + CharSequence charSeq = (CharSequence) value; + return JsonNodeFactory.instance.textNode(charSeq.toString()); + case BYTES: + if (value instanceof byte[]) + return JsonNodeFactory.instance.binaryNode((byte[]) value); + else if (value instanceof ByteBuffer) + return JsonNodeFactory.instance.binaryNode(((ByteBuffer) value).array()); + else + throw new DataException("Invalid type for bytes type: " + value.getClass()); + case ARRAY: { + Collection collection = (Collection) value; + ArrayNode list = JsonNodeFactory.instance.arrayNode(); + for (Object elem : collection) { + Schema valueSchema = schema == null ? null : schema.valueSchema(); + JsonNode fieldValue = convertToJson(valueSchema, elem); + list.add(fieldValue); + } + return list; + } + case MAP: { + Map map = (Map) value; + // If true, using string keys and JSON object; if false, using non-string keys and Array-encoding + boolean objectMode; + if (schema == null) { + objectMode = true; + for (Map.Entry entry : map.entrySet()) { + if (!(entry.getKey() instanceof String)) { + objectMode = false; + break; + } + } + } else { + objectMode = schema.keySchema().type() == Schema.Type.STRING; + } + ObjectNode obj = null; + ArrayNode list = null; + if (objectMode) + obj = JsonNodeFactory.instance.objectNode(); + else + list = JsonNodeFactory.instance.arrayNode(); + for (Map.Entry entry : map.entrySet()) { + Schema keySchema = schema == null ? null : schema.keySchema(); + Schema valueSchema = schema == null ? null : schema.valueSchema(); + JsonNode mapKey = convertToJson(keySchema, entry.getKey()); + JsonNode mapValue = convertToJson(valueSchema, entry.getValue()); + + if (objectMode) + obj.set(mapKey.asText(), mapValue); + else + list.add(JsonNodeFactory.instance.arrayNode().add(mapKey).add(mapValue)); + } + return objectMode ? obj : list; + } + case STRUCT: { + Struct struct = (Struct) value; + if (struct.schema() != schema) + throw new DataException("Mismatching schema."); + ObjectNode obj = JsonNodeFactory.instance.objectNode(); + for (Field field : schema.fields()) { + obj.set(field.name(), convertToJson(field.schema(), struct.get(field))); + } + return obj; + } + } + + throw new DataException("Couldn't convert " + value + " to JSON."); + } catch (ClassCastException e) { + throw new DataException("Invalid type for " + schema.type() + ": " + value.getClass()); + } + } + + + private static Object convertToCopycat(Schema schema, JsonNode jsonValue) { + JsonToCopycatTypeConverter typeConverter; + final Schema.Type schemaType; + if (schema != null) { + schemaType = schema.type(); + } else { + switch (jsonValue.getNodeType()) { + case NULL: + // Special case. With no schema + return null; + case BOOLEAN: + schemaType = Schema.Type.BOOLEAN; + break; + case NUMBER: + if (jsonValue.isIntegralNumber()) + schemaType = Schema.Type.INT64; + else + schemaType = Schema.Type.FLOAT64; + break; + case ARRAY: + schemaType = Schema.Type.ARRAY; + break; + case OBJECT: + schemaType = Schema.Type.MAP; + break; + case STRING: + schemaType = Schema.Type.STRING; + break; + + case BINARY: + case MISSING: + case POJO: + default: + schemaType = null; + break; + } + } + typeConverter = TO_COPYCAT_CONVERTERS.get(schemaType); + if (typeConverter == null) + throw new DataException("Unknown schema type: " + schema.type()); + + Object converted = typeConverter.convert(schema, jsonValue); + if (schema != null && schema.name() != null) { + LogicalTypeConverter logicalConverter = TO_COPYCAT_LOGICAL_CONVERTERS.get(schema.name()); + if (logicalConverter != null) + converted = logicalConverter.convert(schema, converted); + } + return converted; + } + + + private interface JsonToCopycatTypeConverter { + Object convert(Schema schema, JsonNode value); + } + + private interface LogicalTypeConverter { + Object convert(Schema schema, Object value); + } +} diff --git a/copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonDeserializer.java b/copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonDeserializer.java new file mode 100644 index 0000000000000..1661754408697 --- /dev/null +++ b/copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonDeserializer.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.copycat.json; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.serialization.Deserializer; + +import java.util.Map; + +/** + * JSON deserializer for Jackson's JsonNode tree model. Using the tree model allows it to work with arbitrarily + * structured data without having associated Java classes. This deserializer also supports Copycat schemas. + */ +public class JsonDeserializer implements Deserializer { + private ObjectMapper objectMapper = new ObjectMapper(); + + /** + * Default constructor needed by Kafka + */ + public JsonDeserializer() { + } + + @Override + public void configure(Map props, boolean isKey) { + } + + @Override + public JsonNode deserialize(String topic, byte[] bytes) { + if (bytes == null) + return null; + + JsonNode data; + try { + data = objectMapper.readTree(bytes); + } catch (Exception e) { + throw new SerializationException(e); + } + + return data; + } + + @Override + public void close() { + + } +} diff --git a/copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonSchema.java b/copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonSchema.java new file mode 100644 index 0000000000000..78712f3c14ac3 --- /dev/null +++ b/copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonSchema.java @@ -0,0 +1,82 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.json; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import com.fasterxml.jackson.databind.node.ObjectNode; + +public class JsonSchema { + + static final String ENVELOPE_SCHEMA_FIELD_NAME = "schema"; + static final String ENVELOPE_PAYLOAD_FIELD_NAME = "payload"; + static final String SCHEMA_TYPE_FIELD_NAME = "type"; + static final String SCHEMA_OPTIONAL_FIELD_NAME = "optional"; + static final String SCHEMA_NAME_FIELD_NAME = "name"; + static final String SCHEMA_VERSION_FIELD_NAME = "version"; + static final String SCHEMA_DOC_FIELD_NAME = "doc"; + static final String SCHEMA_PARAMETERS_FIELD_NAME = "parameters"; + static final String SCHEMA_DEFAULT_FIELD_NAME = "default"; + static final String ARRAY_ITEMS_FIELD_NAME = "items"; + static final String MAP_KEY_FIELD_NAME = "keys"; + static final String MAP_VALUE_FIELD_NAME = "values"; + static final String STRUCT_FIELDS_FIELD_NAME = "fields"; + static final String STRUCT_FIELD_NAME_FIELD_NAME = "field"; + static final String BOOLEAN_TYPE_NAME = "boolean"; + static final ObjectNode BOOLEAN_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, BOOLEAN_TYPE_NAME); + static final String INT8_TYPE_NAME = "int8"; + static final ObjectNode INT8_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, INT8_TYPE_NAME); + static final String INT16_TYPE_NAME = "int16"; + static final ObjectNode INT16_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, INT16_TYPE_NAME); + static final String INT32_TYPE_NAME = "int32"; + static final ObjectNode INT32_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, INT32_TYPE_NAME); + static final String INT64_TYPE_NAME = "int64"; + static final ObjectNode INT64_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, INT64_TYPE_NAME); + static final String FLOAT_TYPE_NAME = "float"; + static final ObjectNode FLOAT_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, FLOAT_TYPE_NAME); + static final String DOUBLE_TYPE_NAME = "double"; + static final ObjectNode DOUBLE_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, DOUBLE_TYPE_NAME); + static final String BYTES_TYPE_NAME = "bytes"; + static final ObjectNode BYTES_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, BYTES_TYPE_NAME); + static final String STRING_TYPE_NAME = "string"; + static final ObjectNode STRING_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, STRING_TYPE_NAME); + static final String ARRAY_TYPE_NAME = "array"; + static final String MAP_TYPE_NAME = "map"; + static final String STRUCT_TYPE_NAME = "struct"; + + public static ObjectNode envelope(JsonNode schema, JsonNode payload) { + ObjectNode result = JsonNodeFactory.instance.objectNode(); + result.set(ENVELOPE_SCHEMA_FIELD_NAME, schema); + result.set(ENVELOPE_PAYLOAD_FIELD_NAME, payload); + return result; + } + + static class Envelope { + public JsonNode schema; + public JsonNode payload; + + public Envelope(JsonNode schema, JsonNode payload) { + this.schema = schema; + this.payload = payload; + } + + public ObjectNode toJsonNode() { + return envelope(schema, payload); + } + } +} diff --git a/copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonSerializer.java b/copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonSerializer.java new file mode 100644 index 0000000000000..129d14b3d2831 --- /dev/null +++ b/copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonSerializer.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.copycat.json; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.serialization.Serializer; + +import java.util.Map; + +/** + * Serialize Jackson JsonNode tree model objects to UTF-8 JSON. Using the tree model allows handling arbitrarily + * structured data without corresponding Java classes. This serializer also supports Copycat schemas. + */ +public class JsonSerializer implements Serializer { + private final ObjectMapper objectMapper = new ObjectMapper(); + + /** + * Default constructor needed by Kafka + */ + public JsonSerializer() { + + } + + @Override + public void configure(Map config, boolean isKey) { + } + + @Override + public byte[] serialize(String topic, JsonNode data) { + if (data == null) + return null; + + try { + return objectMapper.writeValueAsBytes(data); + } catch (Exception e) { + throw new SerializationException("Error serializing JSON message", e); + } + } + + @Override + public void close() { + } + +} diff --git a/copycat/json/src/test/java/org/apache/kafka/copycat/json/JsonConverterTest.java b/copycat/json/src/test/java/org/apache/kafka/copycat/json/JsonConverterTest.java new file mode 100644 index 0000000000000..6b40046799ae6 --- /dev/null +++ b/copycat/json/src/test/java/org/apache/kafka/copycat/json/JsonConverterTest.java @@ -0,0 +1,644 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.json; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import org.apache.kafka.copycat.data.Date; +import org.apache.kafka.copycat.data.Decimal; +import com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.kafka.common.cache.Cache; +import org.apache.kafka.copycat.data.Schema; +import org.apache.kafka.copycat.data.SchemaAndValue; +import org.apache.kafka.copycat.data.SchemaBuilder; +import org.apache.kafka.copycat.data.Struct; +import org.apache.kafka.copycat.data.Time; +import org.apache.kafka.copycat.data.Timestamp; +import org.apache.kafka.copycat.errors.DataException; +import org.junit.Before; +import org.junit.Test; +import org.powermock.reflect.Whitebox; + +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Calendar; +import java.util.Collections; +import java.util.GregorianCalendar; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.TimeZone; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class JsonConverterTest { + private static final String TOPIC = "topic"; + + ObjectMapper objectMapper = new ObjectMapper(); + JsonConverter converter = new JsonConverter(); + + @Before + public void setUp() { + converter.configure(Collections.EMPTY_MAP, false); + } + + // Schema metadata + + @Test + public void testCopycatSchemaMetadataTranslation() { + // this validates the non-type fields are translated and handled properly + assertEquals(new SchemaAndValue(Schema.BOOLEAN_SCHEMA, true), converter.toCopycatData(TOPIC, "{ \"schema\": { \"type\": \"boolean\" }, \"payload\": true }".getBytes())); + assertEquals(new SchemaAndValue(Schema.OPTIONAL_BOOLEAN_SCHEMA, null), converter.toCopycatData(TOPIC, "{ \"schema\": { \"type\": \"boolean\", \"optional\": true }, \"payload\": null }".getBytes())); + assertEquals(new SchemaAndValue(SchemaBuilder.bool().defaultValue(true).build(), true), + converter.toCopycatData(TOPIC, "{ \"schema\": { \"type\": \"boolean\", \"default\": true }, \"payload\": null }".getBytes())); + assertEquals(new SchemaAndValue(SchemaBuilder.bool().required().name("bool").version(2).doc("the documentation").parameter("foo", "bar").build(), true), + converter.toCopycatData(TOPIC, "{ \"schema\": { \"type\": \"boolean\", \"optional\": false, \"name\": \"bool\", \"version\": 2, \"doc\": \"the documentation\", \"parameters\": { \"foo\": \"bar\" }}, \"payload\": true }".getBytes())); + } + + // Schema types + + @Test + public void booleanToCopycat() { + assertEquals(new SchemaAndValue(Schema.BOOLEAN_SCHEMA, true), converter.toCopycatData(TOPIC, "{ \"schema\": { \"type\": \"boolean\" }, \"payload\": true }".getBytes())); + assertEquals(new SchemaAndValue(Schema.BOOLEAN_SCHEMA, false), converter.toCopycatData(TOPIC, "{ \"schema\": { \"type\": \"boolean\" }, \"payload\": false }".getBytes())); + } + + @Test + public void byteToCopycat() { + assertEquals(new SchemaAndValue(Schema.INT8_SCHEMA, (byte) 12), converter.toCopycatData(TOPIC, "{ \"schema\": { \"type\": \"int8\" }, \"payload\": 12 }".getBytes())); + } + + @Test + public void shortToCopycat() { + assertEquals(new SchemaAndValue(Schema.INT16_SCHEMA, (short) 12), converter.toCopycatData(TOPIC, "{ \"schema\": { \"type\": \"int16\" }, \"payload\": 12 }".getBytes())); + } + + @Test + public void intToCopycat() { + assertEquals(new SchemaAndValue(Schema.INT32_SCHEMA, 12), converter.toCopycatData(TOPIC, "{ \"schema\": { \"type\": \"int32\" }, \"payload\": 12 }".getBytes())); + } + + @Test + public void longToCopycat() { + assertEquals(new SchemaAndValue(Schema.INT64_SCHEMA, 12L), converter.toCopycatData(TOPIC, "{ \"schema\": { \"type\": \"int64\" }, \"payload\": 12 }".getBytes())); + assertEquals(new SchemaAndValue(Schema.INT64_SCHEMA, 4398046511104L), converter.toCopycatData(TOPIC, "{ \"schema\": { \"type\": \"int64\" }, \"payload\": 4398046511104 }".getBytes())); + } + + @Test + public void floatToCopycat() { + assertEquals(new SchemaAndValue(Schema.FLOAT32_SCHEMA, 12.34f), converter.toCopycatData(TOPIC, "{ \"schema\": { \"type\": \"float\" }, \"payload\": 12.34 }".getBytes())); + } + + @Test + public void doubleToCopycat() { + assertEquals(new SchemaAndValue(Schema.FLOAT64_SCHEMA, 12.34), converter.toCopycatData(TOPIC, "{ \"schema\": { \"type\": \"double\" }, \"payload\": 12.34 }".getBytes())); + } + + + @Test + public void bytesToCopycat() throws UnsupportedEncodingException { + ByteBuffer reference = ByteBuffer.wrap("test-string".getBytes("UTF-8")); + String msg = "{ \"schema\": { \"type\": \"bytes\" }, \"payload\": \"dGVzdC1zdHJpbmc=\" }"; + SchemaAndValue schemaAndValue = converter.toCopycatData(TOPIC, msg.getBytes()); + ByteBuffer converted = ByteBuffer.wrap((byte[]) schemaAndValue.value()); + assertEquals(reference, converted); + } + + @Test + public void stringToCopycat() { + assertEquals(new SchemaAndValue(Schema.STRING_SCHEMA, "foo-bar-baz"), converter.toCopycatData(TOPIC, "{ \"schema\": { \"type\": \"string\" }, \"payload\": \"foo-bar-baz\" }".getBytes())); + } + + @Test + public void arrayToCopycat() { + byte[] arrayJson = "{ \"schema\": { \"type\": \"array\", \"items\": { \"type\" : \"int32\" } }, \"payload\": [1, 2, 3] }".getBytes(); + assertEquals(new SchemaAndValue(SchemaBuilder.array(Schema.INT32_SCHEMA).build(), Arrays.asList(1, 2, 3)), converter.toCopycatData(TOPIC, arrayJson)); + } + + @Test + public void mapToCopycatStringKeys() { + byte[] mapJson = "{ \"schema\": { \"type\": \"map\", \"keys\": { \"type\" : \"string\" }, \"values\": { \"type\" : \"int32\" } }, \"payload\": { \"key1\": 12, \"key2\": 15} }".getBytes(); + Map expected = new HashMap<>(); + expected.put("key1", 12); + expected.put("key2", 15); + assertEquals(new SchemaAndValue(SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.INT32_SCHEMA).build(), expected), converter.toCopycatData(TOPIC, mapJson)); + } + + @Test + public void mapToCopycatNonStringKeys() { + byte[] mapJson = "{ \"schema\": { \"type\": \"map\", \"keys\": { \"type\" : \"int32\" }, \"values\": { \"type\" : \"int32\" } }, \"payload\": [ [1, 12], [2, 15] ] }".getBytes(); + Map expected = new HashMap<>(); + expected.put(1, 12); + expected.put(2, 15); + assertEquals(new SchemaAndValue(SchemaBuilder.map(Schema.INT32_SCHEMA, Schema.INT32_SCHEMA).build(), expected), converter.toCopycatData(TOPIC, mapJson)); + } + + @Test + public void structToCopycat() { + byte[] structJson = "{ \"schema\": { \"type\": \"struct\", \"fields\": [{ \"field\": \"field1\", \"type\": \"boolean\" }, { \"field\": \"field2\", \"type\": \"string\" }] }, \"payload\": { \"field1\": true, \"field2\": \"string\" } }".getBytes(); + Schema expectedSchema = SchemaBuilder.struct().field("field1", Schema.BOOLEAN_SCHEMA).field("field2", Schema.STRING_SCHEMA).build(); + Struct expected = new Struct(expectedSchema).put("field1", true).put("field2", "string"); + SchemaAndValue converted = converter.toCopycatData(TOPIC, structJson); + assertEquals(new SchemaAndValue(expectedSchema, expected), converted); + } + + @Test(expected = DataException.class) + public void nullToCopycat() { + // When schemas are enabled, trying to decode a null should be an error -- we should *always* have the envelope + assertEquals(SchemaAndValue.NULL, converter.toCopycatData(TOPIC, null)); + } + + @Test + public void nullSchemaPrimitiveToCopycat() { + SchemaAndValue converted = converter.toCopycatData(TOPIC, "{ \"schema\": null, \"payload\": null }".getBytes()); + assertEquals(SchemaAndValue.NULL, converted); + + converted = converter.toCopycatData(TOPIC, "{ \"schema\": null, \"payload\": true }".getBytes()); + assertEquals(new SchemaAndValue(null, true), converted); + + // Integers: Copycat has more data types, and JSON unfortunately mixes all number types. We try to preserve + // info as best we can, so we always use the largest integer and floating point numbers we can and have Jackson + // determine if it's an integer or not + converted = converter.toCopycatData(TOPIC, "{ \"schema\": null, \"payload\": 12 }".getBytes()); + assertEquals(new SchemaAndValue(null, 12L), converted); + + converted = converter.toCopycatData(TOPIC, "{ \"schema\": null, \"payload\": 12.24 }".getBytes()); + assertEquals(new SchemaAndValue(null, 12.24), converted); + + converted = converter.toCopycatData(TOPIC, "{ \"schema\": null, \"payload\": \"a string\" }".getBytes()); + assertEquals(new SchemaAndValue(null, "a string"), converted); + + converted = converter.toCopycatData(TOPIC, "{ \"schema\": null, \"payload\": [1, \"2\", 3] }".getBytes()); + assertEquals(new SchemaAndValue(null, Arrays.asList(1L, "2", 3L)), converted); + + converted = converter.toCopycatData(TOPIC, "{ \"schema\": null, \"payload\": { \"field1\": 1, \"field2\": 2} }".getBytes()); + Map obj = new HashMap<>(); + obj.put("field1", 1L); + obj.put("field2", 2L); + assertEquals(new SchemaAndValue(null, obj), converted); + } + + @Test + public void decimalToCopycat() { + Schema schema = Decimal.schema(2); + BigDecimal reference = new BigDecimal(new BigInteger("156"), 2); + // Payload is base64 encoded byte[]{0, -100}, which is the two's complement encoding of 156. + String msg = "{ \"schema\": { \"type\": \"bytes\", \"name\": \"org.apache.kafka.copycat.data.Decimal\", \"version\": 1, \"parameters\": { \"scale\": \"2\" } }, \"payload\": \"AJw=\" }"; + SchemaAndValue schemaAndValue = converter.toCopycatData(TOPIC, msg.getBytes()); + BigDecimal converted = (BigDecimal) schemaAndValue.value(); + assertEquals(schema, schemaAndValue.schema()); + assertEquals(reference, converted); + } + + @Test + public void dateToCopycat() { + Schema schema = Date.SCHEMA; + GregorianCalendar calendar = new GregorianCalendar(1970, Calendar.JANUARY, 1, 0, 0, 0); + calendar.setTimeZone(TimeZone.getTimeZone("UTC")); + calendar.add(Calendar.DATE, 10000); + java.util.Date reference = calendar.getTime(); + String msg = "{ \"schema\": { \"type\": \"int32\", \"name\": \"org.apache.kafka.copycat.data.Date\", \"version\": 1 }, \"payload\": 10000 }"; + SchemaAndValue schemaAndValue = converter.toCopycatData(TOPIC, msg.getBytes()); + java.util.Date converted = (java.util.Date) schemaAndValue.value(); + assertEquals(schema, schemaAndValue.schema()); + assertEquals(reference, converted); + } + + @Test + public void timeToCopycat() { + Schema schema = Time.SCHEMA; + GregorianCalendar calendar = new GregorianCalendar(1970, Calendar.JANUARY, 1, 0, 0, 0); + calendar.setTimeZone(TimeZone.getTimeZone("UTC")); + calendar.add(Calendar.MILLISECOND, 14400000); + java.util.Date reference = calendar.getTime(); + String msg = "{ \"schema\": { \"type\": \"int32\", \"name\": \"org.apache.kafka.copycat.data.Time\", \"version\": 1 }, \"payload\": 14400000 }"; + SchemaAndValue schemaAndValue = converter.toCopycatData(TOPIC, msg.getBytes()); + java.util.Date converted = (java.util.Date) schemaAndValue.value(); + assertEquals(schema, schemaAndValue.schema()); + assertEquals(reference, converted); + } + + @Test + public void timestampToCopycat() { + Schema schema = Timestamp.SCHEMA; + GregorianCalendar calendar = new GregorianCalendar(1970, Calendar.JANUARY, 1, 0, 0, 0); + calendar.setTimeZone(TimeZone.getTimeZone("UTC")); + calendar.add(Calendar.MILLISECOND, 2000000000); + calendar.add(Calendar.MILLISECOND, 2000000000); + java.util.Date reference = calendar.getTime(); + String msg = "{ \"schema\": { \"type\": \"int64\", \"name\": \"org.apache.kafka.copycat.data.Timestamp\", \"version\": 1 }, \"payload\": 4000000000 }"; + SchemaAndValue schemaAndValue = converter.toCopycatData(TOPIC, msg.getBytes()); + java.util.Date converted = (java.util.Date) schemaAndValue.value(); + assertEquals(schema, schemaAndValue.schema()); + assertEquals(reference, converted); + } + + // Schema metadata + + @Test + public void testJsonSchemaMetadataTranslation() { + JsonNode converted = parse(converter.fromCopycatData(TOPIC, Schema.BOOLEAN_SCHEMA, true)); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"boolean\", \"optional\": false }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertEquals(true, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).booleanValue()); + + converted = parse(converter.fromCopycatData(TOPIC, Schema.OPTIONAL_BOOLEAN_SCHEMA, null)); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"boolean\", \"optional\": true }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertTrue(converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).isNull()); + + converted = parse(converter.fromCopycatData(TOPIC, SchemaBuilder.bool().defaultValue(true).build(), true)); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"boolean\", \"optional\": false, \"default\": true }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertEquals(true, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).booleanValue()); + + converted = parse(converter.fromCopycatData(TOPIC, SchemaBuilder.bool().required().name("bool").version(3).doc("the documentation").parameter("foo", "bar").build(), true)); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"boolean\", \"optional\": false, \"name\": \"bool\", \"version\": 3, \"doc\": \"the documentation\", \"parameters\": { \"foo\": \"bar\" }}"), + converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertEquals(true, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).booleanValue()); + } + + + @Test + public void testCacheSchemaToCopycatConversion() { + Cache cache = Whitebox.getInternalState(converter, "toCopycatSchemaCache"); + assertEquals(0, cache.size()); + + converter.toCopycatData(TOPIC, "{ \"schema\": { \"type\": \"boolean\" }, \"payload\": true }".getBytes()); + assertEquals(1, cache.size()); + + converter.toCopycatData(TOPIC, "{ \"schema\": { \"type\": \"boolean\" }, \"payload\": true }".getBytes()); + assertEquals(1, cache.size()); + + // Different schema should also get cached + converter.toCopycatData(TOPIC, "{ \"schema\": { \"type\": \"boolean\", \"optional\": true }, \"payload\": true }".getBytes()); + assertEquals(2, cache.size()); + + // Even equivalent, but different JSON encoding of schema, should get different cache entry + converter.toCopycatData(TOPIC, "{ \"schema\": { \"type\": \"boolean\", \"optional\": false }, \"payload\": true }".getBytes()); + assertEquals(3, cache.size()); + } + + // Schema types + + @Test + public void booleanToJson() { + JsonNode converted = parse(converter.fromCopycatData(TOPIC, Schema.BOOLEAN_SCHEMA, true)); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"boolean\", \"optional\": false }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertEquals(true, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).booleanValue()); + } + + @Test + public void byteToJson() { + JsonNode converted = parse(converter.fromCopycatData(TOPIC, Schema.INT8_SCHEMA, (byte) 12)); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"int8\", \"optional\": false }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertEquals(12, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).intValue()); + } + + @Test + public void shortToJson() { + JsonNode converted = parse(converter.fromCopycatData(TOPIC, Schema.INT16_SCHEMA, (short) 12)); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"int16\", \"optional\": false }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertEquals(12, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).intValue()); + } + + @Test + public void intToJson() { + JsonNode converted = parse(converter.fromCopycatData(TOPIC, Schema.INT32_SCHEMA, 12)); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"int32\", \"optional\": false }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertEquals(12, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).intValue()); + } + + @Test + public void longToJson() { + JsonNode converted = parse(converter.fromCopycatData(TOPIC, Schema.INT64_SCHEMA, 4398046511104L)); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"int64\", \"optional\": false }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertEquals(4398046511104L, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).longValue()); + } + + @Test + public void floatToJson() { + JsonNode converted = parse(converter.fromCopycatData(TOPIC, Schema.FLOAT32_SCHEMA, 12.34f)); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"float\", \"optional\": false }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertEquals(12.34f, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).floatValue(), 0.001); + } + + @Test + public void doubleToJson() { + JsonNode converted = parse(converter.fromCopycatData(TOPIC, Schema.FLOAT64_SCHEMA, 12.34)); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"double\", \"optional\": false }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertEquals(12.34, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).doubleValue(), 0.001); + } + + @Test + public void bytesToJson() throws IOException { + JsonNode converted = parse(converter.fromCopycatData(TOPIC, Schema.BYTES_SCHEMA, "test-string".getBytes())); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"bytes\", \"optional\": false }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertEquals(ByteBuffer.wrap("test-string".getBytes()), + ByteBuffer.wrap(converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).binaryValue())); + } + + @Test + public void stringToJson() { + JsonNode converted = parse(converter.fromCopycatData(TOPIC, Schema.STRING_SCHEMA, "test-string")); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"string\", \"optional\": false }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertEquals("test-string", converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).textValue()); + } + + @Test + public void arrayToJson() { + Schema int32Array = SchemaBuilder.array(Schema.INT32_SCHEMA).build(); + JsonNode converted = parse(converter.fromCopycatData(TOPIC, int32Array, Arrays.asList(1, 2, 3))); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"array\", \"items\": { \"type\": \"int32\", \"optional\": false }, \"optional\": false }"), + converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertEquals(JsonNodeFactory.instance.arrayNode().add(1).add(2).add(3), + converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME)); + } + + @Test + public void mapToJsonStringKeys() { + Schema stringIntMap = SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.INT32_SCHEMA).build(); + Map input = new HashMap<>(); + input.put("key1", 12); + input.put("key2", 15); + JsonNode converted = parse(converter.fromCopycatData(TOPIC, stringIntMap, input)); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"map\", \"keys\": { \"type\" : \"string\", \"optional\": false }, \"values\": { \"type\" : \"int32\", \"optional\": false }, \"optional\": false }"), + converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertEquals(JsonNodeFactory.instance.objectNode().put("key1", 12).put("key2", 15), + converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME)); + } + + @Test + public void mapToJsonNonStringKeys() { + Schema intIntMap = SchemaBuilder.map(Schema.INT32_SCHEMA, Schema.INT32_SCHEMA).build(); + Map input = new HashMap<>(); + input.put(1, 12); + input.put(2, 15); + JsonNode converted = parse(converter.fromCopycatData(TOPIC, intIntMap, input)); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"map\", \"keys\": { \"type\" : \"int32\", \"optional\": false }, \"values\": { \"type\" : \"int32\", \"optional\": false }, \"optional\": false }"), + converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + + assertTrue(converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).isArray()); + ArrayNode payload = (ArrayNode) converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME); + assertEquals(2, payload.size()); + Set payloadEntries = new HashSet<>(); + for (JsonNode elem : payload) + payloadEntries.add(elem); + assertEquals(new HashSet<>(Arrays.asList(JsonNodeFactory.instance.arrayNode().add(1).add(12), + JsonNodeFactory.instance.arrayNode().add(2).add(15))), + payloadEntries + ); + } + + @Test + public void structToJson() { + Schema schema = SchemaBuilder.struct().field("field1", Schema.BOOLEAN_SCHEMA).field("field2", Schema.STRING_SCHEMA).build(); + Struct input = new Struct(schema).put("field1", true).put("field2", "string"); + JsonNode converted = parse(converter.fromCopycatData(TOPIC, schema, input)); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"struct\", \"optional\": false, \"fields\": [{ \"field\": \"field1\", \"type\": \"boolean\", \"optional\": false }, { \"field\": \"field2\", \"type\": \"string\", \"optional\": false }] }"), + converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertEquals(JsonNodeFactory.instance.objectNode() + .put("field1", true) + .put("field2", "string"), + converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME)); + } + + + @Test + public void decimalToJson() throws IOException { + JsonNode converted = parse(converter.fromCopycatData(TOPIC, Decimal.schema(2), new BigDecimal(new BigInteger("156"), 2))); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"bytes\", \"optional\": false, \"name\": \"org.apache.kafka.copycat.data.Decimal\", \"version\": 1, \"parameters\": { \"scale\": \"2\" } }"), + converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertArrayEquals(new byte[]{0, -100}, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).binaryValue()); + } + + @Test + public void dateToJson() throws IOException { + GregorianCalendar calendar = new GregorianCalendar(1970, Calendar.JANUARY, 1, 0, 0, 0); + calendar.setTimeZone(TimeZone.getTimeZone("UTC")); + calendar.add(Calendar.DATE, 10000); + java.util.Date date = calendar.getTime(); + + JsonNode converted = parse(converter.fromCopycatData(TOPIC, Date.SCHEMA, date)); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"int32\", \"optional\": false, \"name\": \"org.apache.kafka.copycat.data.Date\", \"version\": 1 }"), + converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + JsonNode payload = converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME); + assertTrue(payload.isInt()); + assertEquals(10000, payload.intValue()); + } + + @Test + public void timeToJson() throws IOException { + GregorianCalendar calendar = new GregorianCalendar(1970, Calendar.JANUARY, 1, 0, 0, 0); + calendar.setTimeZone(TimeZone.getTimeZone("UTC")); + calendar.add(Calendar.MILLISECOND, 14400000); + java.util.Date date = calendar.getTime(); + + JsonNode converted = parse(converter.fromCopycatData(TOPIC, Time.SCHEMA, date)); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"int32\", \"optional\": false, \"name\": \"org.apache.kafka.copycat.data.Time\", \"version\": 1 }"), + converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + JsonNode payload = converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME); + assertTrue(payload.isInt()); + assertEquals(14400000, payload.longValue()); + } + + @Test + public void timestampToJson() throws IOException { + GregorianCalendar calendar = new GregorianCalendar(1970, Calendar.JANUARY, 1, 0, 0, 0); + calendar.setTimeZone(TimeZone.getTimeZone("UTC")); + calendar.add(Calendar.MILLISECOND, 2000000000); + calendar.add(Calendar.MILLISECOND, 2000000000); + java.util.Date date = calendar.getTime(); + + JsonNode converted = parse(converter.fromCopycatData(TOPIC, Timestamp.SCHEMA, date)); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"int64\", \"optional\": false, \"name\": \"org.apache.kafka.copycat.data.Timestamp\", \"version\": 1 }"), + converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + JsonNode payload = converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME); + assertTrue(payload.isLong()); + assertEquals(4000000000L, payload.longValue()); + } + + + @Test + public void nullSchemaAndPrimitiveToJson() { + // This still needs to do conversion of data, null schema means "anything goes" + JsonNode converted = parse(converter.fromCopycatData(TOPIC, null, true)); + validateEnvelopeNullSchema(converted); + assertTrue(converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME).isNull()); + assertEquals(true, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).booleanValue()); + } + + @Test + public void nullSchemaAndArrayToJson() { + // This still needs to do conversion of data, null schema means "anything goes". Make sure we mix and match + // types to verify conversion still works. + JsonNode converted = parse(converter.fromCopycatData(TOPIC, null, Arrays.asList(1, "string", true))); + validateEnvelopeNullSchema(converted); + assertTrue(converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME).isNull()); + assertEquals(JsonNodeFactory.instance.arrayNode().add(1).add("string").add(true), + converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME)); + } + + @Test + public void nullSchemaAndMapToJson() { + // This still needs to do conversion of data, null schema means "anything goes". Make sure we mix and match + // types to verify conversion still works. + Map input = new HashMap<>(); + input.put("key1", 12); + input.put("key2", "string"); + input.put("key3", true); + JsonNode converted = parse(converter.fromCopycatData(TOPIC, null, input)); + validateEnvelopeNullSchema(converted); + assertTrue(converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME).isNull()); + assertEquals(JsonNodeFactory.instance.objectNode().put("key1", 12).put("key2", "string").put("key3", true), + converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME)); + } + + @Test + public void nullSchemaAndMapNonStringKeysToJson() { + // This still needs to do conversion of data, null schema means "anything goes". Make sure we mix and match + // types to verify conversion still works. + Map input = new HashMap<>(); + input.put("string", 12); + input.put(52, "string"); + input.put(false, true); + JsonNode converted = parse(converter.fromCopycatData(TOPIC, null, input)); + validateEnvelopeNullSchema(converted); + assertTrue(converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME).isNull()); + assertTrue(converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).isArray()); + ArrayNode payload = (ArrayNode) converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME); + assertEquals(3, payload.size()); + Set payloadEntries = new HashSet<>(); + for (JsonNode elem : payload) + payloadEntries.add(elem); + assertEquals(new HashSet<>(Arrays.asList(JsonNodeFactory.instance.arrayNode().add("string").add(12), + JsonNodeFactory.instance.arrayNode().add(52).add("string"), + JsonNodeFactory.instance.arrayNode().add(false).add(true))), + payloadEntries + ); + } + + + @Test(expected = DataException.class) + public void mismatchSchemaJson() { + // If we have mismatching schema info, we should properly convert to a DataException + converter.fromCopycatData(TOPIC, Schema.FLOAT64_SCHEMA, true); + } + + + + @Test + public void noSchemaToCopycat() { + Map props = Collections.singletonMap("schemas.enable", false); + converter.configure(props, true); + assertEquals(new SchemaAndValue(null, true), converter.toCopycatData(TOPIC, "true".getBytes())); + } + + @Test + public void noSchemaToJson() { + Map props = Collections.singletonMap("schemas.enable", false); + converter.configure(props, true); + JsonNode converted = parse(converter.fromCopycatData(TOPIC, null, true)); + assertTrue(converted.isBoolean()); + assertEquals(true, converted.booleanValue()); + } + + @Test + public void testCacheSchemaToJsonConversion() { + Cache cache = Whitebox.getInternalState(converter, "fromCopycatSchemaCache"); + assertEquals(0, cache.size()); + + // Repeated conversion of the same schema, even if the schema object is different should return the same Java + // object + converter.fromCopycatData(TOPIC, SchemaBuilder.bool().build(), true); + assertEquals(1, cache.size()); + + converter.fromCopycatData(TOPIC, SchemaBuilder.bool().build(), true); + assertEquals(1, cache.size()); + + // Validate that a similar, but different schema correctly returns a different schema. + converter.fromCopycatData(TOPIC, SchemaBuilder.bool().optional().build(), true); + assertEquals(2, cache.size()); + } + + + private JsonNode parse(byte[] json) { + try { + return objectMapper.readTree(json); + } catch (IOException e) { + fail("IOException during JSON parse: " + e.getMessage()); + throw new RuntimeException("failed"); + } + } + + private JsonNode parse(String json) { + try { + return objectMapper.readTree(json); + } catch (IOException e) { + fail("IOException during JSON parse: " + e.getMessage()); + throw new RuntimeException("failed"); + } + } + + private void validateEnvelope(JsonNode env) { + assertNotNull(env); + assertTrue(env.isObject()); + assertEquals(2, env.size()); + assertTrue(env.has(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertTrue(env.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME).isObject()); + assertTrue(env.has(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME)); + } + + private void validateEnvelopeNullSchema(JsonNode env) { + assertNotNull(env); + assertTrue(env.isObject()); + assertEquals(2, env.size()); + assertTrue(env.has(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertTrue(env.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME).isNull()); + assertTrue(env.has(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME)); + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/cli/CopycatDistributed.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/cli/CopycatDistributed.java new file mode 100644 index 0000000000000..b5e88964b559e --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/cli/CopycatDistributed.java @@ -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 org.apache.kafka.copycat.cli; + +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.copycat.runtime.Copycat; +import org.apache.kafka.copycat.runtime.Herder; +import org.apache.kafka.copycat.runtime.Worker; +import org.apache.kafka.copycat.runtime.standalone.StandaloneHerder; +import org.apache.kafka.copycat.storage.KafkaOffsetBackingStore; +import org.apache.kafka.copycat.util.Callback; +import org.apache.kafka.copycat.util.FutureCallback; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.Properties; + +/** + *

            + * Command line utility that runs Copycat in distributed mode. In this mode, the process joints a group of other workers + * and work is distributed among them. This is useful for running Copycat as a service, where connectors can be + * submitted to the cluster to be automatically executed in a scalable, distributed fashion. This also allows you to + * easily scale out horizontally, elastically adding or removing capacity simply by starting or stopping worker + * instances. + *

            + */ +@InterfaceStability.Unstable +public class CopycatDistributed { + private static final Logger log = LoggerFactory.getLogger(CopycatDistributed.class); + + public static void main(String[] args) throws Exception { + Properties workerProps; + Properties connectorProps; + + if (args.length < 2) { + log.info("Usage: CopycatDistributed worker.properties connector1.properties [connector2.properties ...]"); + System.exit(1); + } + + String workerPropsFile = args[0]; + workerProps = !workerPropsFile.isEmpty() ? Utils.loadProps(workerPropsFile) : new Properties(); + + WorkerConfig workerConfig = new WorkerConfig(workerProps); + Worker worker = new Worker(workerConfig, new KafkaOffsetBackingStore()); + Herder herder = new StandaloneHerder(worker); + final Copycat copycat = new Copycat(worker, herder); + copycat.start(); + + try { + for (final String connectorPropsFile : Arrays.copyOfRange(args, 1, args.length)) { + connectorProps = Utils.loadProps(connectorPropsFile); + FutureCallback cb = new FutureCallback<>(new Callback() { + @Override + public void onCompletion(Throwable error, String id) { + if (error != null) + log.error("Failed to create job for {}", connectorPropsFile); + } + }); + herder.addConnector(connectorProps, cb); + cb.get(); + } + } catch (Throwable t) { + log.error("Stopping after connector error", t); + copycat.stop(); + } + + // Shutdown will be triggered by Ctrl-C or via HTTP shutdown request + copycat.awaitStop(); + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/cli/CopycatStandalone.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/cli/CopycatStandalone.java new file mode 100644 index 0000000000000..12ec154b6a8a6 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/cli/CopycatStandalone.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.copycat.cli; + +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.copycat.runtime.Copycat; +import org.apache.kafka.copycat.runtime.Herder; +import org.apache.kafka.copycat.runtime.Worker; +import org.apache.kafka.copycat.runtime.standalone.StandaloneHerder; +import org.apache.kafka.copycat.storage.FileOffsetBackingStore; +import org.apache.kafka.copycat.util.Callback; +import org.apache.kafka.copycat.util.FutureCallback; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.Properties; + +/** + *

            + * Command line utility that runs Copycat as a standalone process. In this mode, work is not + * distributed. Instead, all the normal Copycat machinery works within a single process. This is + * useful for ad hoc, small, or experimental jobs. + *

            + *

            + * By default, no job configs or offset data is persistent. You can make jobs persistent and + * fault tolerant by overriding the settings to use file storage for both. + *

            + */ +@InterfaceStability.Unstable +public class CopycatStandalone { + private static final Logger log = LoggerFactory.getLogger(CopycatStandalone.class); + + public static void main(String[] args) throws Exception { + Properties workerProps; + Properties connectorProps; + + if (args.length < 2) { + log.info("Usage: CopycatStandalone worker.properties connector1.properties [connector2.properties ...]"); + System.exit(1); + } + + String workerPropsFile = args[0]; + workerProps = !workerPropsFile.isEmpty() ? Utils.loadProps(workerPropsFile) : new Properties(); + + WorkerConfig workerConfig = new WorkerConfig(workerProps); + Worker worker = new Worker(workerConfig, new FileOffsetBackingStore()); + Herder herder = new StandaloneHerder(worker); + final Copycat copycat = new Copycat(worker, herder); + copycat.start(); + + try { + for (final String connectorPropsFile : Arrays.copyOfRange(args, 1, args.length)) { + connectorProps = Utils.loadProps(connectorPropsFile); + FutureCallback cb = new FutureCallback<>(new Callback() { + @Override + public void onCompletion(Throwable error, String id) { + if (error != null) + log.error("Failed to create job for {}", connectorPropsFile); + } + }); + herder.addConnector(connectorProps, cb); + cb.get(); + } + } catch (Throwable t) { + log.error("Stopping after connector error", t); + copycat.stop(); + } + + // Shutdown will be triggered by Ctrl-C or via HTTP shutdown request + copycat.awaitStop(); + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/cli/WorkerConfig.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/cli/WorkerConfig.java new file mode 100644 index 0000000000000..a976d9059c9e0 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/cli/WorkerConfig.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.copycat.cli; + +import org.apache.kafka.common.annotation.InterfaceStability; +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 java.util.Properties; + +/** + * Configuration for standalone workers. + */ +@InterfaceStability.Unstable +public class WorkerConfig extends AbstractConfig { + + public static final String CLUSTER_CONFIG = "cluster"; + private static final String CLUSTER_CONFIG_DOC = + "ID for this cluster, which is used to provide a namespace so multiple Copycat clusters " + + "or instances may co-exist while sharing a single Kafka cluster."; + public static final String CLUSTER_DEFAULT = "copycat"; + + public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers"; + public static final String BOOTSTRAP_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 BOOTSTRAP_SERVERS_DEFAULT = "localhost:9092"; + + public static final String KEY_CONVERTER_CLASS_CONFIG = "key.converter"; + public static final String KEY_CONVERTER_CLASS_DOC = + "Converter class for key Copycat data that implements the Converter interface."; + + public static final String VALUE_CONVERTER_CLASS_CONFIG = "value.converter"; + public static final String VALUE_CONVERTER_CLASS_DOC = + "Converter class for value Copycat data that implements the Converter interface."; + + public static final String OFFSET_KEY_CONVERTER_CLASS_CONFIG = "offset.key.converter"; + public static final String OFFSET_KEY_CONVERTER_CLASS_DOC = + "Converter class for offset key Copycat data that implements the Converter interface."; + + public static final String OFFSET_VALUE_CONVERTER_CLASS_CONFIG = "offset.value.converter"; + public static final String OFFSET_VALUE_CONVERTER_CLASS_DOC = + "Converter class for offset value Copycat data that implements the Converter interface."; + + public static final String TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_CONFIG + = "task.shutdown.graceful.timeout.ms"; + private static final String TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_DOC = + "Amount of time to wait for tasks to shutdown gracefully. This is the total amount of time," + + " not per task. All task have shutdown triggered, then they are waited on sequentially."; + private static final String TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_DEFAULT = "5000"; + + public static final String OFFSET_COMMIT_INTERVAL_MS_CONFIG = "offset.flush.interval.ms"; + private static final String OFFSET_COMMIT_INTERVAL_MS_DOC + = "Interval at which to try committing offsets for tasks."; + public static final long OFFSET_COMMIT_INTERVAL_MS_DEFAULT = 60000L; + + public static final String OFFSET_COMMIT_TIMEOUT_MS_CONFIG = "offset.flush.timeout.ms"; + private static final String OFFSET_COMMIT_TIMEOUT_MS_DOC + = "Maximum number of milliseconds to wait for records to flush and partition offset data to be" + + " committed to offset storage before cancelling the process and restoring the offset " + + "data to be committed in a future attempt."; + public static final long OFFSET_COMMIT_TIMEOUT_MS_DEFAULT = 5000L; + + private static ConfigDef config; + + static { + config = new ConfigDef() + .define(CLUSTER_CONFIG, Type.STRING, CLUSTER_DEFAULT, Importance.HIGH, CLUSTER_CONFIG_DOC) + .define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, BOOTSTRAP_SERVERS_DEFAULT, + Importance.HIGH, BOOTSTRAP_SERVERS_DOC) + .define(KEY_CONVERTER_CLASS_CONFIG, Type.CLASS, + Importance.HIGH, KEY_CONVERTER_CLASS_DOC) + .define(VALUE_CONVERTER_CLASS_CONFIG, Type.CLASS, + Importance.HIGH, VALUE_CONVERTER_CLASS_DOC) + .define(OFFSET_KEY_CONVERTER_CLASS_CONFIG, Type.CLASS, + Importance.HIGH, OFFSET_KEY_CONVERTER_CLASS_DOC) + .define(OFFSET_VALUE_CONVERTER_CLASS_CONFIG, Type.CLASS, + Importance.HIGH, OFFSET_VALUE_CONVERTER_CLASS_DOC) + .define(TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_CONFIG, Type.LONG, + TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_DEFAULT, Importance.LOW, + TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_DOC) + .define(OFFSET_COMMIT_INTERVAL_MS_CONFIG, Type.LONG, OFFSET_COMMIT_INTERVAL_MS_DEFAULT, + Importance.LOW, OFFSET_COMMIT_INTERVAL_MS_DOC) + .define(OFFSET_COMMIT_TIMEOUT_MS_CONFIG, Type.LONG, OFFSET_COMMIT_TIMEOUT_MS_DEFAULT, + Importance.LOW, OFFSET_COMMIT_TIMEOUT_MS_DOC); + } + + public WorkerConfig() { + this(new Properties()); + } + + public WorkerConfig(Properties props) { + super(config, props); + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/ConnectorConfig.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/ConnectorConfig.java new file mode 100644 index 0000000000000..336597edbed4c --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/ConnectorConfig.java @@ -0,0 +1,73 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

            + * http://www.apache.org/licenses/LICENSE-2.0 + *

            + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.runtime; + +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 java.util.Properties; + +/** + *

            + * Configuration options for Connectors. These only include Copycat system-level configuration + * options (e.g. Connector class name, timeouts used by Copycat to control the connector) but does + * not include Connector-specific options (e.g. database connection settings). + *

            + *

            + * Note that some of these options are not required for all connectors. For example TOPICS_CONFIG + * is sink-specific. + *

            + */ +public class ConnectorConfig extends AbstractConfig { + + public static final String NAME_CONFIG = "name"; + private static final String NAME_DOC = "Globally unique name to use for this connector."; + + public static final String CONNECTOR_CLASS_CONFIG = "connector.class"; + private static final String CONNECTOR_CLASS_DOC = + "Name of the class for this connector. Must be a subclass of org.apache.kafka.copycat.connector" + + ".Connector"; + + public static final String TASKS_MAX_CONFIG = "tasks.max"; + private static final String TASKS_MAX_DOC = "Maximum number of tasks to use for this connector."; + public static final int TASKS_MAX_DEFAULT = 1; + + public static final String TOPICS_CONFIG = "topics"; + private static final String TOPICS_DOC = ""; + public static final String TOPICS_DEFAULT = ""; + + private static ConfigDef config; + + static { + config = new ConfigDef() + .define(NAME_CONFIG, Type.STRING, Importance.HIGH, NAME_DOC) + .define(CONNECTOR_CLASS_CONFIG, Type.STRING, Importance.HIGH, CONNECTOR_CLASS_DOC) + .define(TASKS_MAX_CONFIG, Type.INT, TASKS_MAX_DEFAULT, Importance.HIGH, TASKS_MAX_DOC) + .define(TOPICS_CONFIG, Type.LIST, TOPICS_DEFAULT, Importance.HIGH, TOPICS_DOC); + } + + public ConnectorConfig() { + this(new Properties()); + } + + public ConnectorConfig(Properties props) { + super(config, props); + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/Copycat.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/Copycat.java new file mode 100644 index 0000000000000..e8dfe1477edef --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/Copycat.java @@ -0,0 +1,94 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

            + * http://www.apache.org/licenses/LICENSE-2.0 + *

            + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.runtime; + +import org.apache.kafka.common.annotation.InterfaceStability; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * This class ties together all the components of a Copycat process (herder, worker, + * storage, command interface), managing their lifecycle. + */ +@InterfaceStability.Unstable +public class Copycat { + private static final Logger log = LoggerFactory.getLogger(Copycat.class); + + private final Worker worker; + private final Herder herder; + private final CountDownLatch startLatch = new CountDownLatch(1); + private final CountDownLatch stopLatch = new CountDownLatch(1); + private final AtomicBoolean shutdown = new AtomicBoolean(false); + private final ShutdownHook shutdownHook; + + public Copycat(Worker worker, Herder herder) { + log.debug("Copycat created"); + this.worker = worker; + this.herder = herder; + shutdownHook = new ShutdownHook(); + } + + public void start() { + log.info("Copycat starting"); + Runtime.getRuntime().addShutdownHook(shutdownHook); + + worker.start(); + herder.start(); + + log.info("Copycat started"); + + startLatch.countDown(); + } + + public void stop() { + boolean wasShuttingDown = shutdown.getAndSet(true); + if (!wasShuttingDown) { + log.info("Copycat stopping"); + + herder.stop(); + worker.stop(); + + log.info("Copycat stopped"); + } + + stopLatch.countDown(); + } + + public void awaitStop() { + try { + stopLatch.await(); + } catch (InterruptedException e) { + log.error("Interrupted waiting for Copycat to shutdown"); + } + } + + private class ShutdownHook extends Thread { + @Override + public void run() { + try { + startLatch.await(); + Copycat.this.stop(); + } catch (InterruptedException e) { + log.error("Interrupted in shutdown hook while waiting for copycat startup to finish"); + } + } + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/Herder.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/Herder.java new file mode 100644 index 0000000000000..7f8b7c22b404b --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/Herder.java @@ -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 org.apache.kafka.copycat.runtime; + +import org.apache.kafka.copycat.util.Callback; + +import java.util.Properties; + +/** + *

            + * The herder interface tracks and manages workers and connectors. It is the main interface for external components + * to make changes to the state of the cluster. For example, in distributed mode, an implementation of this class + * knows how to accept a connector configuration, may need to route it to the current leader worker for the cluster so + * the config can be written to persistent storage, and then ensures the new connector is correctly instantiated on one + * of the workers. + *

            + *

            + * This class must implement all the actions that can be taken on the cluster (add/remove connectors, pause/resume tasks, + * get state of connectors and tasks, etc). The non-Java interfaces to the cluster (REST API and CLI) are very simple + * wrappers of the functionality provided by this interface. + *

            + *

            + * In standalone mode, this implementation of this class will be trivial because no coordination is needed. In that case, + * the implementation will mainly be delegating tasks directly to other components. For example, when creating a new + * connector in standalone mode, there is no need to persist the config and the connector and its tasks must run in the + * same process, so the standalone herder implementation can immediately instantiate and start the connector and its + * tasks. + *

            + */ +public interface Herder { + + void start(); + + void stop(); + + /** + * Submit a connector job to the cluster. This works from any node by forwarding the request to + * the leader herder if necessary. + * + * @param connectorProps user-specified properties for this job + * @param callback callback to invoke when the request completes + */ + void addConnector(Properties connectorProps, Callback callback); + + /** + * Delete a connector job by name. + * + * @param name name of the connector job to shutdown and delete + * @param callback callback to invoke when the request completes + */ + void deleteConnector(String name, Callback callback); +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/SourceTaskOffsetCommitter.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/SourceTaskOffsetCommitter.java new file mode 100644 index 0000000000000..953cfa5f148fd --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/SourceTaskOffsetCommitter.java @@ -0,0 +1,103 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

            + * http://www.apache.org/licenses/LICENSE-2.0 + *

            + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.runtime; + +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.copycat.cli.WorkerConfig; +import org.apache.kafka.copycat.util.ConnectorTaskId; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +/** + *

            + * Manages offset commit scheduling and execution for SourceTasks. + *

            + *

            + * Unlike sink tasks which directly manage their offset commits in the main poll() thread since + * they drive the event loop and control (for all intents and purposes) the timeouts, source + * tasks are at the whim of the connector and cannot be guaranteed to wake up on the necessary + * schedule. Instead, this class tracks all the active tasks, their schedule for commits, and + * ensures they are invoked in a timely fashion. + *

            + *

            + * The current implementation uses a single thread to process commits and + *

            + */ +class SourceTaskOffsetCommitter { + private static final Logger log = LoggerFactory.getLogger(SourceTaskOffsetCommitter.class); + + private Time time; + private WorkerConfig config; + private ScheduledExecutorService commitExecutorService = null; + private HashMap> committers = new HashMap<>(); + + SourceTaskOffsetCommitter(Time time, WorkerConfig config) { + this.time = time; + this.config = config; + commitExecutorService = Executors.newSingleThreadScheduledExecutor(); + } + + public void close(long timeoutMs) { + commitExecutorService.shutdown(); + try { + if (!commitExecutorService.awaitTermination(timeoutMs, TimeUnit.MILLISECONDS)) { + log.error("Graceful shutdown of offset commitOffsets thread timed out."); + } + } catch (InterruptedException e) { + // ignore and allow to exit immediately + } + } + + public void schedule(final ConnectorTaskId id, final WorkerSourceTask workerTask) { + long commitIntervalMs = config.getLong(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG); + ScheduledFuture commitFuture = commitExecutorService.scheduleWithFixedDelay(new Runnable() { + @Override + public void run() { + commit(workerTask); + } + }, commitIntervalMs, commitIntervalMs, TimeUnit.MILLISECONDS); + committers.put(id, commitFuture); + } + + public void remove(ConnectorTaskId id) { + ScheduledFuture commitFuture = committers.remove(id); + commitFuture.cancel(false); + } + + public void commit(WorkerSourceTask workerTask) { + try { + log.debug("Committing offsets for {}", workerTask); + boolean success = workerTask.commitOffsets(); + if (!success) { + log.error("Failed to commit offsets for {}", workerTask); + } + } catch (Throwable t) { + // We're very careful about exceptions here since any uncaught exceptions in the commit + // thread would cause the fixed interval schedule on the ExecutorService to stop running + // for that task + log.error("Unhandled exception when committing {}: ", workerTask, t); + } + } + +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/Worker.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/Worker.java new file mode 100644 index 0000000000000..a34a014ce93d1 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/Worker.java @@ -0,0 +1,204 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

            + * http://www.apache.org/licenses/LICENSE-2.0 + *

            + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.runtime; + +import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.copycat.cli.WorkerConfig; +import org.apache.kafka.copycat.connector.Task; +import org.apache.kafka.copycat.errors.CopycatException; +import org.apache.kafka.copycat.sink.SinkTask; +import org.apache.kafka.copycat.source.SourceTask; +import org.apache.kafka.copycat.storage.*; +import org.apache.kafka.copycat.util.ConnectorTaskId; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +/** + *

            + * Worker runs a (dynamic) set of tasks in a set of threads, doing the work of actually moving + * data to/from Kafka. + *

            + *

            + * Since each task has a dedicated thread, this is mainly just a container for them. + *

            + */ +public class Worker { + private static final Logger log = LoggerFactory.getLogger(Worker.class); + + private Time time; + private WorkerConfig config; + private Converter keyConverter; + private Converter valueConverter; + private Converter offsetKeyConverter; + private Converter offsetValueConverter; + private OffsetBackingStore offsetBackingStore; + private HashMap tasks = new HashMap<>(); + private KafkaProducer producer; + private SourceTaskOffsetCommitter sourceTaskOffsetCommitter; + + public Worker(WorkerConfig config, OffsetBackingStore offsetBackingStore) { + this(new SystemTime(), config, offsetBackingStore); + } + + @SuppressWarnings("unchecked") + public Worker(Time time, WorkerConfig config, OffsetBackingStore offsetBackingStore) { + this.time = time; + this.config = config; + this.keyConverter = config.getConfiguredInstance(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, Converter.class); + this.keyConverter.configure(config.originalsWithPrefix("key.converter."), true); + this.valueConverter = config.getConfiguredInstance(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, Converter.class); + this.valueConverter.configure(config.originalsWithPrefix("value.converter."), false); + this.offsetKeyConverter = config.getConfiguredInstance(WorkerConfig.OFFSET_KEY_CONVERTER_CLASS_CONFIG, Converter.class); + this.offsetKeyConverter.configure(config.originalsWithPrefix("offset.key.converter."), true); + this.offsetValueConverter = config.getConfiguredInstance(WorkerConfig.OFFSET_VALUE_CONVERTER_CLASS_CONFIG, Converter.class); + this.offsetValueConverter.configure(config.originalsWithPrefix("offset.value.converter."), false); + + this.offsetBackingStore = offsetBackingStore; + this.offsetBackingStore.configure(config.originals()); + } + + public void start() { + log.info("Worker starting"); + + Properties unusedConfigs = config.unusedProperties(); + + Map producerProps = new HashMap<>(); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, Utils.join(config.getList(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG), ",")); + 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"); + for (String propName : unusedConfigs.stringPropertyNames()) { + producerProps.put(propName, unusedConfigs.getProperty(propName)); + } + producer = new KafkaProducer<>(producerProps); + + offsetBackingStore.start(); + sourceTaskOffsetCommitter = new SourceTaskOffsetCommitter(time, config); + + log.info("Worker started"); + } + + public void stop() { + log.info("Worker stopping"); + + long started = time.milliseconds(); + long limit = started + config.getLong(WorkerConfig.TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_CONFIG); + + for (Map.Entry entry : tasks.entrySet()) { + WorkerTask task = entry.getValue(); + log.warn("Shutting down task {} uncleanly; herder should have shut down " + + "tasks before the Worker is stopped.", task); + try { + task.stop(); + } catch (CopycatException e) { + log.error("Error while shutting down task " + task, e); + } + } + + for (Map.Entry entry : tasks.entrySet()) { + WorkerTask task = entry.getValue(); + log.debug("Waiting for task {} to finish shutting down", task); + if (!task.awaitStop(Math.max(limit - time.milliseconds(), 0))) + log.error("Graceful shutdown of task {} failed.", task); + task.close(); + } + + long timeoutMs = limit - time.milliseconds(); + sourceTaskOffsetCommitter.close(timeoutMs); + + offsetBackingStore.stop(); + + log.info("Worker stopped"); + } + + /** + * Add a new task. + * @param id Globally unique ID for this task. + * @param taskClassName name of the {@link org.apache.kafka.copycat.connector.Task} + * class to instantiate. Must be a subclass of either + * {@link org.apache.kafka.copycat.source.SourceTask} or + * {@link org.apache.kafka.copycat.sink.SinkTask}. + * @param props configuration options for the task + */ + public void addTask(ConnectorTaskId id, String taskClassName, Properties props) { + if (tasks.containsKey(id)) { + String msg = "Task already exists in this worker; the herder should not have requested " + + "that this : " + id; + log.error(msg); + throw new CopycatException(msg); + } + + final Task task = instantiateTask(taskClassName); + + // Decide which type of worker task we need based on the type of task. + final WorkerTask workerTask; + if (task instanceof SourceTask) { + SourceTask sourceTask = (SourceTask) task; + OffsetStorageReader offsetReader = new OffsetStorageReaderImpl(offsetBackingStore, id.connector(), + offsetKeyConverter, offsetValueConverter); + OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetBackingStore, id.connector(), + offsetKeyConverter, offsetValueConverter); + workerTask = new WorkerSourceTask(id, sourceTask, keyConverter, valueConverter, producer, + offsetReader, offsetWriter, config, time); + } else if (task instanceof SinkTask) { + workerTask = new WorkerSinkTask(id, (SinkTask) task, config, keyConverter, valueConverter, time); + } else { + log.error("Tasks must be a subclass of either SourceTask or SinkTask", task); + throw new CopycatException("Tasks must be a subclass of either SourceTask or SinkTask"); + } + + // Start the task before adding modifying any state, any exceptions are caught higher up the + // call chain and there's no cleanup to do here + workerTask.start(props); + tasks.put(id, workerTask); + } + + private static Task instantiateTask(String taskClassName) { + try { + return Utils.newInstance(Class.forName(taskClassName).asSubclass(Task.class)); + } catch (ClassNotFoundException e) { + throw new CopycatException("Task class not found", e); + } + } + + public void stopTask(ConnectorTaskId id) { + WorkerTask task = getTask(id); + task.stop(); + if (!task.awaitStop(config.getLong(WorkerConfig.TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_CONFIG))) + log.error("Graceful stop of task {} failed.", task); + task.close(); + tasks.remove(id); + } + + private WorkerTask getTask(ConnectorTaskId id) { + WorkerTask task = tasks.get(id); + if (task == null) { + log.error("Task not found: " + id); + throw new CopycatException("Task not found: " + id); + } + return task; + } + +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerSinkTask.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerSinkTask.java new file mode 100644 index 0000000000000..edb415a4f77f6 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerSinkTask.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.copycat.runtime; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.clients.consumer.*; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.copycat.cli.WorkerConfig; +import org.apache.kafka.copycat.data.SchemaAndValue; +import org.apache.kafka.copycat.errors.CopycatException; +import org.apache.kafka.copycat.errors.IllegalWorkerStateException; +import org.apache.kafka.copycat.sink.SinkRecord; +import org.apache.kafka.copycat.sink.SinkTask; +import org.apache.kafka.copycat.sink.SinkTaskContext; +import org.apache.kafka.copycat.storage.Converter; +import org.apache.kafka.copycat.util.ConnectorTaskId; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.*; +import java.util.concurrent.TimeUnit; + +/** + * WorkerTask that uses a SinkTask to export data from Kafka. + */ +class WorkerSinkTask implements WorkerTask { + private static final Logger log = LoggerFactory.getLogger(WorkerSinkTask.class); + + private final ConnectorTaskId id; + private final SinkTask task; + private final WorkerConfig workerConfig; + private final Time time; + private final Converter keyConverter; + private final Converter valueConverter; + private WorkerSinkTaskThread workThread; + private KafkaConsumer consumer; + private final SinkTaskContext context; + + public WorkerSinkTask(ConnectorTaskId id, SinkTask task, WorkerConfig workerConfig, + Converter keyConverter, Converter valueConverter, Time time) { + this.id = id; + this.task = task; + this.workerConfig = workerConfig; + this.keyConverter = keyConverter; + this.valueConverter = valueConverter; + this.time = time; + this.context = new WorkerSinkTaskContext(); + } + + @Override + public void start(Properties props) { + task.initialize(context); + task.start(props); + consumer = createConsumer(props); + workThread = createWorkerThread(); + workThread.start(); + } + + @Override + public void stop() { + // Offset commit is handled upon exit in work thread + task.stop(); + if (workThread != null) + workThread.startGracefulShutdown(); + consumer.wakeup(); + } + + @Override + public boolean awaitStop(long timeoutMs) { + if (workThread != null) { + try { + boolean success = workThread.awaitShutdown(timeoutMs, TimeUnit.MILLISECONDS); + if (!success) + workThread.forceShutdown(); + return success; + } catch (InterruptedException e) { + return false; + } + } + return true; + } + + @Override + public void close() { + // FIXME Kafka needs to add a timeout parameter here for us to properly obey the timeout + // passed in + if (consumer != null) + consumer.close(); + } + + /** Poll for new messages with the given timeout. Should only be invoked by the worker thread. */ + public void poll(long timeoutMs) { + try { + log.trace("{} polling consumer with timeout {} ms", id, timeoutMs); + ConsumerRecords msgs = consumer.poll(timeoutMs); + log.trace("{} polling returned {} messages", id, msgs.count()); + deliverMessages(msgs); + } catch (ConsumerWakeupException we) { + log.trace("{} consumer woken up", id); + } + } + + /** + * Starts an offset commit by flushing outstanding messages from the task and then starting + * the write commit. This should only be invoked by the WorkerSinkTaskThread. + **/ + public void commitOffsets(long now, boolean sync, final int seqno, boolean flush) { + log.info("{} Committing offsets", this); + HashMap offsets = new HashMap<>(); + for (TopicPartition tp : consumer.assignment()) { + offsets.put(tp, new OffsetAndMetadata(consumer.position(tp))); + } + // We only don't flush the task in one case: when shutting down, the task has already been + // stopped and all data should have already been flushed + if (flush) { + try { + task.flush(offsets); + } catch (Throwable t) { + log.error("Commit of {} offsets failed due to exception while flushing: {}", this, t); + workThread.onCommitCompleted(t, seqno); + return; + } + } + + if (sync) { + try { + consumer.commitSync(offsets); + } catch (KafkaException e) { + workThread.onCommitCompleted(e, seqno); + } + } else { + OffsetCommitCallback cb = new OffsetCommitCallback() { + @Override + public void onComplete(Map offsets, Exception error) { + workThread.onCommitCompleted(error, seqno); + } + }; + consumer.commitAsync(offsets, cb); + } + } + + public Time time() { + return time; + } + + public WorkerConfig workerConfig() { + return workerConfig; + } + + private KafkaConsumer createConsumer(Properties taskProps) { + String topicsStr = taskProps.getProperty(SinkTask.TOPICS_CONFIG); + if (topicsStr == null || topicsStr.isEmpty()) + throw new CopycatException("Sink tasks require a list of topics."); + String[] topics = topicsStr.split(","); + + // Include any unknown worker configs so consumer configs can be set globally on the worker + // and through to the task + Properties props = workerConfig.unusedProperties(); + props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "copycat-" + id.toString()); + props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, + Utils.join(workerConfig.getList(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG), ",")); + props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + props.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + props.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + props.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + + KafkaConsumer newConsumer; + try { + newConsumer = new KafkaConsumer<>(props); + } catch (Throwable t) { + throw new CopycatException("Failed to create consumer", t); + } + + log.debug("Task {} subscribing to topics {}", id, topics); + newConsumer.subscribe(Arrays.asList(topics)); + + // Seek to any user-provided offsets. This is useful if offsets are tracked in the downstream system (e.g., to + // enable exactly once delivery to that system). + // + // To do this correctly, we need to first make sure we have been assigned partitions, which poll() will guarantee. + // We ask for offsets after this poll to make sure any offsets committed before the rebalance are picked up correctly. + newConsumer.poll(0); + Map offsets = context.offsets(); + for (TopicPartition tp : newConsumer.assignment()) { + Long offset = offsets.get(tp); + if (offset != null) + newConsumer.seek(tp, offset); + } + return newConsumer; + } + + private WorkerSinkTaskThread createWorkerThread() { + return new WorkerSinkTaskThread(this, "WorkerSinkTask-" + id, time, workerConfig); + } + + private void deliverMessages(ConsumerRecords msgs) { + // Finally, deliver this batch to the sink + if (msgs.count() > 0) { + List records = new ArrayList<>(); + for (ConsumerRecord msg : msgs) { + log.trace("Consuming message with key {}, value {}", msg.key(), msg.value()); + SchemaAndValue keyAndSchema = keyConverter.toCopycatData(msg.topic(), msg.key()); + SchemaAndValue valueAndSchema = valueConverter.toCopycatData(msg.topic(), msg.value()); + records.add( + new SinkRecord(msg.topic(), msg.partition(), + keyAndSchema.schema(), keyAndSchema.value(), + valueAndSchema.schema(), valueAndSchema.value(), + msg.offset()) + ); + } + + try { + task.put(records); + } catch (CopycatException e) { + log.error("Exception from SinkTask {}: ", id, e); + } catch (Throwable t) { + log.error("Unexpected exception from SinkTask {}: ", id, t); + } + } + } + + + private class WorkerSinkTaskContext extends SinkTaskContext { + @Override + public Set assignment() { + if (consumer == null) + throw new IllegalWorkerStateException("SinkTaskContext may not be used to look up partition assignment until the task is initialized"); + return consumer.assignment(); + } + + @Override + public void pause(TopicPartition... partitions) { + if (consumer == null) + throw new IllegalWorkerStateException("SinkTaskContext may not be used to pause consumption until the task is initialized"); + try { + consumer.pause(partitions); + } catch (IllegalStateException e) { + throw new IllegalWorkerStateException("SinkTasks may not pause partitions that are not currently assigned to them.", e); + } + } + + @Override + public void resume(TopicPartition... partitions) { + if (consumer == null) + throw new IllegalWorkerStateException("SinkTaskContext may not be used to resume consumption until the task is initialized"); + try { + consumer.resume(partitions); + } catch (IllegalStateException e) { + throw new IllegalWorkerStateException("SinkTasks may not resume partitions that are not currently assigned to them.", e); + } + } + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerSinkTaskThread.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerSinkTaskThread.java new file mode 100644 index 0000000000000..41e38f0e34d1b --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerSinkTaskThread.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.copycat.runtime; + +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.copycat.cli.WorkerConfig; +import org.apache.kafka.copycat.util.ShutdownableThread; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Worker thread for a WorkerSinkTask. These classes are very tightly coupled, but separated to + * simplify testing. + */ +class WorkerSinkTaskThread extends ShutdownableThread { + private static final Logger log = LoggerFactory.getLogger(WorkerSinkTask.class); + + private final WorkerSinkTask task; + private long nextCommit; + private boolean committing; + private int commitSeqno; + private long commitStarted; + private int commitFailures; + + public WorkerSinkTaskThread(WorkerSinkTask task, String name, Time time, + WorkerConfig workerConfig) { + super(name); + this.task = task; + this.nextCommit = time.milliseconds() + + workerConfig.getLong(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG); + this.committing = false; + this.commitSeqno = 0; + this.commitStarted = -1; + this.commitFailures = 0; + } + + @Override + public void execute() { + while (getRunning()) { + iteration(); + } + // Make sure any uncommitted data has committed + task.commitOffsets(task.time().milliseconds(), true, -1, false); + } + + public void iteration() { + long now = task.time().milliseconds(); + + // Maybe commit + if (!committing && now >= nextCommit) { + synchronized (this) { + committing = true; + commitSeqno += 1; + commitStarted = now; + } + task.commitOffsets(now, false, commitSeqno, true); + nextCommit += task.workerConfig().getLong(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG); + } + + // Check for timed out commits + long commitTimeout = commitStarted + task.workerConfig().getLong( + WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_CONFIG); + if (committing && now >= commitTimeout) { + log.warn("Commit of {} offsets timed out", this); + commitFailures++; + committing = false; + } + + // And process messages + long timeoutMs = Math.max(nextCommit - now, 0); + task.poll(timeoutMs); + } + + public void onCommitCompleted(Throwable error, long seqno) { + synchronized (this) { + if (commitSeqno != seqno) { + log.debug("Got callback for timed out commit {}: {}, but most recent commit is {}", + this, + seqno, commitSeqno); + } else { + if (error != null) { + log.error("Commit of {} offsets threw an unexpected exception: ", this, error); + commitFailures++; + } else { + log.debug("Finished {} offset commit successfully in {} ms", + this, task.time().milliseconds() - commitStarted); + commitFailures = 0; + } + committing = false; + } + } + } + + public int commitFailures() { + return commitFailures; + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerSourceTask.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerSourceTask.java new file mode 100644 index 0000000000000..ee0a532fb305c --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerSourceTask.java @@ -0,0 +1,307 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

            + * http://www.apache.org/licenses/LICENSE-2.0 + *

            + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.runtime; + +import org.apache.kafka.common.utils.Time; +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; +import org.apache.kafka.copycat.cli.WorkerConfig; +import org.apache.kafka.copycat.source.SourceRecord; +import org.apache.kafka.copycat.source.SourceTask; +import org.apache.kafka.copycat.source.SourceTaskContext; +import org.apache.kafka.copycat.storage.Converter; +import org.apache.kafka.copycat.storage.OffsetStorageReader; +import org.apache.kafka.copycat.storage.OffsetStorageWriter; +import org.apache.kafka.copycat.util.ConnectorTaskId; +import org.apache.kafka.copycat.util.ShutdownableThread; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.IdentityHashMap; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +/** + * WorkerTask that uses a SourceTask to ingest data into Kafka. + */ +class WorkerSourceTask implements WorkerTask { + private static final Logger log = LoggerFactory.getLogger(WorkerSourceTask.class); + + private ConnectorTaskId id; + private SourceTask task; + private final Converter keyConverter; + private final Converter valueConverter; + private KafkaProducer producer; + private WorkerSourceTaskThread workThread; + private OffsetStorageReader offsetReader; + private OffsetStorageWriter offsetWriter; + private final WorkerConfig workerConfig; + private final Time time; + + // Use IdentityHashMap to ensure correctness with duplicate records. This is a HashMap because + // there is no IdentityHashSet. + private IdentityHashMap, ProducerRecord> outstandingMessages; + // A second buffer is used while an offset flush is running + private IdentityHashMap, ProducerRecord> outstandingMessagesBacklog; + private boolean flushing; + + public WorkerSourceTask(ConnectorTaskId id, SourceTask task, + Converter keyConverter, Converter valueConverter, + KafkaProducer producer, + OffsetStorageReader offsetReader, OffsetStorageWriter offsetWriter, + WorkerConfig workerConfig, Time time) { + this.id = id; + this.task = task; + this.keyConverter = keyConverter; + this.valueConverter = valueConverter; + this.producer = producer; + this.offsetReader = offsetReader; + this.offsetWriter = offsetWriter; + this.workerConfig = workerConfig; + this.time = time; + + this.outstandingMessages = new IdentityHashMap<>(); + this.outstandingMessagesBacklog = new IdentityHashMap<>(); + this.flushing = false; + } + + @Override + public void start(Properties props) { + task.initialize(new SourceTaskContext(offsetReader)); + task.start(props); + workThread = new WorkerSourceTaskThread("WorkerSourceTask-" + id); + workThread.start(); + } + + @Override + public void stop() { + task.stop(); + commitOffsets(); + if (workThread != null) + workThread.startGracefulShutdown(); + } + + @Override + public boolean awaitStop(long timeoutMs) { + if (workThread != null) { + try { + boolean success = workThread.awaitShutdown(timeoutMs, TimeUnit.MILLISECONDS); + if (!success) + workThread.forceShutdown(); + return success; + } catch (InterruptedException e) { + return false; + } + } + return true; + } + + @Override + public void close() { + // Nothing to do + } + + /** + * Send a batch of records. This is atomic up to the point of getting the messages into the + * Producer and recorded in our set of outstanding messages, so either all or none will be sent + * @param records + */ + private synchronized void sendRecords(List records) { + for (SourceRecord record : records) { + byte[] key = keyConverter.fromCopycatData(record.topic(), record.keySchema(), record.key()); + byte[] value = valueConverter.fromCopycatData(record.topic(), record.valueSchema(), record.value()); + final ProducerRecord producerRecord = new ProducerRecord<>(record.topic(), record.kafkaPartition(), key, value); + log.trace("Appending record with key {}, value {}", record.key(), record.value()); + if (!flushing) { + outstandingMessages.put(producerRecord, producerRecord); + } else { + outstandingMessagesBacklog.put(producerRecord, producerRecord); + } + producer.send( + producerRecord, + new Callback() { + @Override + public void onCompletion(RecordMetadata recordMetadata, Exception e) { + if (e != null) { + log.error("Failed to send record: ", e); + } else { + log.trace("Wrote record successfully: topic {} partition {} offset {}", + recordMetadata.topic(), recordMetadata.partition(), + recordMetadata.offset()); + } + recordSent(producerRecord); + } + }); + // Offsets are converted & serialized in the OffsetWriter + offsetWriter.offset(record.sourcePartition(), record.sourceOffset()); + } + } + + private synchronized void recordSent(final ProducerRecord record) { + ProducerRecord removed = outstandingMessages.remove(record); + // While flushing, we may also see callbacks for items in the backlog + if (removed == null && flushing) + removed = outstandingMessagesBacklog.remove(record); + // But if neither one had it, something is very wrong + if (removed == null) { + log.error("Saw callback for record that was not present in the outstanding message set: " + + "{}", record); + } else if (flushing && outstandingMessages.isEmpty()) { + // flush thread may be waiting on the outstanding messages to clear + this.notifyAll(); + } + } + + public boolean commitOffsets() { + long commitTimeoutMs = workerConfig.getLong(WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_CONFIG); + + long started = time.milliseconds(); + long timeout = started + commitTimeoutMs; + + synchronized (this) { + // First we need to make sure we snapshot everything in exactly the current state. This + // means both the current set of messages we're still waiting to finish, stored in this + // class, which setting flushing = true will handle by storing any new values into a new + // buffer; and the current set of user-specified offsets, stored in the + // OffsetStorageWriter, for which we can use beginFlush() to initiate the snapshot. + flushing = true; + boolean flushStarted = offsetWriter.beginFlush(); + // Still wait for any producer records to flush, even if there aren't any offsets to write + // to persistent storage + + // Next we need to wait for all outstanding messages to finish sending + while (!outstandingMessages.isEmpty()) { + try { + long timeoutMs = timeout - time.milliseconds(); + if (timeoutMs <= 0) { + log.error( + "Failed to flush {}, timed out while waiting for producer to flush outstanding " + + "messages", this.toString()); + finishFailedFlush(); + return false; + } + this.wait(timeoutMs); + } catch (InterruptedException e) { + // ignore + } + } + + if (!flushStarted) { + // There was nothing in the offsets to process, but we still waited for the data in the + // buffer to flush. This is useful since this can feed into metrics to monitor, e.g. + // flush time, which can be used for monitoring even if the connector doesn't record any + // offsets. + finishSuccessfulFlush(); + log.debug("Finished {} offset commitOffsets successfully in {} ms", + this, time.milliseconds() - started); + return true; + } + } + + // Now we can actually flush the offsets to user storage. + Future flushFuture = offsetWriter.doFlush(new org.apache.kafka.copycat.util.Callback() { + @Override + public void onCompletion(Throwable error, Void result) { + if (error != null) { + log.error("Failed to flush {} offsets to storage: ", this, error); + } else { + log.trace("Finished flushing {} offsets to storage", this); + } + } + }); + // Very rare case: offsets were unserializable and we finished immediately, unable to store + // any data + if (flushFuture == null) { + finishFailedFlush(); + return false; + } + try { + flushFuture.get(Math.max(timeout - time.milliseconds(), 0), TimeUnit.MILLISECONDS); + // There's a small race here where we can get the callback just as this times out (and log + // success), but then catch the exception below and cancel everything. This won't cause any + // errors, is only wasteful in this minor edge case, and the worst result is that the log + // could look a little confusing. + } catch (InterruptedException e) { + log.warn("Flush of {} offsets interrupted, cancelling", this); + finishFailedFlush(); + return false; + } catch (ExecutionException e) { + log.error("Flush of {} offsets threw an unexpected exception: ", this, e); + finishFailedFlush(); + return false; + } catch (TimeoutException e) { + log.error("Timed out waiting to flush {} offsets to storage", this); + finishFailedFlush(); + return false; + } + + finishSuccessfulFlush(); + log.debug("Finished {} commitOffsets successfully in {} ms", + this, time.milliseconds() - started); + return true; + } + + private synchronized void finishFailedFlush() { + offsetWriter.cancelFlush(); + outstandingMessages.putAll(outstandingMessagesBacklog); + outstandingMessagesBacklog.clear(); + flushing = false; + } + + private void finishSuccessfulFlush() { + // If we were successful, we can just swap instead of replacing items back into the original map + IdentityHashMap, ProducerRecord> temp = outstandingMessages; + outstandingMessages = outstandingMessagesBacklog; + outstandingMessagesBacklog = temp; + flushing = false; + } + + + private class WorkerSourceTaskThread extends ShutdownableThread { + public WorkerSourceTaskThread(String name) { + super(name); + } + + @Override + public void execute() { + try { + while (getRunning()) { + List records = task.poll(); + if (records == null) + continue; + sendRecords(records); + } + } catch (InterruptedException e) { + // Ignore and allow to exit. + } + } + } + + @Override + public String toString() { + return "WorkerSourceTask{" + + "id=" + id + + '}'; + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerTask.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerTask.java new file mode 100644 index 0000000000000..af225bb7e2d05 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerTask.java @@ -0,0 +1,54 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

            + * http://www.apache.org/licenses/LICENSE-2.0 + *

            + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.runtime; + +import java.util.Properties; + +/** + * Handles processing for an individual task. This interface only provides the basic methods + * used by {@link Worker} to manage the tasks. Implementations combine a user-specified Task with + * Kafka to create a data flow. + */ +interface WorkerTask { + /** + * Start the Task + * @param props initial configuration + */ + void start(Properties props); + + /** + * Stop this task from processing messages. This method does not block, it only triggers + * shutdown. Use #{@link #awaitStop} to block until completion. + */ + void stop(); + + /** + * Wait for this task to finish stopping. + * + * @param timeoutMs + * @return true if successful, false if the timeout was reached + */ + boolean awaitStop(long timeoutMs); + + /** + * Close this task. This is different from #{@link #stop} and #{@link #awaitStop} in that the + * stop methods ensure processing has stopped but may leave resources allocated. This method + * should clean up all resources. + */ + void close(); +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/standalone/StandaloneConnectorContext.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/standalone/StandaloneConnectorContext.java new file mode 100644 index 0000000000000..0e14015158e7c --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/standalone/StandaloneConnectorContext.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.copycat.runtime.standalone; + +import org.apache.kafka.copycat.connector.ConnectorContext; + +/** + * ConnectorContext for use with the StandaloneHerder, which maintains all connectors and tasks + * in a single process. + */ +class StandaloneConnectorContext implements ConnectorContext { + + private StandaloneHerder herder; + private String connectorName; + + public StandaloneConnectorContext(StandaloneHerder herder, String connectorName) { + this.herder = herder; + this.connectorName = connectorName; + } + + @Override + public void requestTaskReconfiguration() { + // This is trivial to forward since there is only one herder and it's in memory in this + // process + herder.requestTaskReconfiguration(connectorName); + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/standalone/StandaloneHerder.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/standalone/StandaloneHerder.java new file mode 100644 index 0000000000000..45d428d063cb6 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/standalone/StandaloneHerder.java @@ -0,0 +1,257 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

            + * http://www.apache.org/licenses/LICENSE-2.0 + *

            + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.runtime.standalone; + +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.copycat.connector.Connector; +import org.apache.kafka.copycat.errors.CopycatException; +import org.apache.kafka.copycat.runtime.ConnectorConfig; +import org.apache.kafka.copycat.runtime.Herder; +import org.apache.kafka.copycat.runtime.Worker; +import org.apache.kafka.copycat.sink.SinkConnector; +import org.apache.kafka.copycat.sink.SinkTask; +import org.apache.kafka.copycat.util.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.*; + +/** + * Single process, in-memory "herder". Useful for a standalone copycat process. + */ +public class StandaloneHerder implements Herder { + private static final Logger log = LoggerFactory.getLogger(StandaloneHerder.class); + + private Worker worker; + private HashMap connectors = new HashMap<>(); + + public StandaloneHerder(Worker worker) { + this.worker = worker; + } + + public synchronized void start() { + log.info("Herder starting"); + log.info("Herder started"); + } + + public synchronized void stop() { + log.info("Herder stopping"); + + // There's no coordination/hand-off to do here since this is all standalone. Instead, we + // should just clean up the stuff we normally would, i.e. cleanly checkpoint and shutdown all + // the tasks. + for (Map.Entry entry : connectors.entrySet()) { + ConnectorState state = entry.getValue(); + stopConnector(state); + } + connectors.clear(); + + log.info("Herder stopped"); + } + + @Override + public synchronized void addConnector(Properties connectorProps, + Callback callback) { + try { + ConnectorState connState = createConnector(connectorProps); + if (callback != null) + callback.onCompletion(null, connState.name); + // This should always be a new job, create jobs from scratch + createConnectorTasks(connState); + } catch (CopycatException e) { + if (callback != null) + callback.onCompletion(e, null); + } + } + + @Override + public synchronized void deleteConnector(String name, Callback callback) { + try { + destroyConnector(name); + if (callback != null) + callback.onCompletion(null, null); + } catch (CopycatException e) { + if (callback != null) + callback.onCompletion(e, null); + } + } + + // Creates the and configures the connector. Does not setup any tasks + private ConnectorState createConnector(Properties connectorProps) { + ConnectorConfig connConfig = new ConnectorConfig(connectorProps); + String connName = connConfig.getString(ConnectorConfig.NAME_CONFIG); + String className = connConfig.getString(ConnectorConfig.CONNECTOR_CLASS_CONFIG); + log.info("Creating connector {} of type {}", connName, className); + int maxTasks = connConfig.getInt(ConnectorConfig.TASKS_MAX_CONFIG); + List topics = connConfig.getList(ConnectorConfig.TOPICS_CONFIG); // Sinks only + Properties configs = connConfig.unusedProperties(); + + if (connectors.containsKey(connName)) { + log.error("Ignoring request to create connector due to conflicting connector name"); + throw new CopycatException("Connector with name " + connName + " already exists"); + } + + final Connector connector; + try { + connector = instantiateConnector(className); + } catch (Throwable t) { + // Catches normal exceptions due to instantiation errors as well as any runtime errors that + // may be caused by user code + throw new CopycatException("Failed to create connector instance", t); + } + connector.initialize(new StandaloneConnectorContext(this, connName)); + try { + connector.start(configs); + } catch (CopycatException e) { + throw new CopycatException("Connector threw an exception while starting", e); + } + ConnectorState state = new ConnectorState(connName, connector, maxTasks, topics); + connectors.put(connName, state); + + log.info("Finished creating connector {}", connName); + + return state; + } + + private static Connector instantiateConnector(String className) { + try { + return Utils.newInstance(className, Connector.class); + } catch (ClassNotFoundException e) { + throw new CopycatException("Couldn't instantiate connector class", e); + } + } + + private void destroyConnector(String connName) { + log.info("Destroying connector {}", connName); + ConnectorState state = connectors.get(connName); + if (state == null) { + log.error("Failed to destroy connector {} because it does not exist", connName); + throw new CopycatException("Connector does not exist"); + } + + stopConnector(state); + connectors.remove(state.name); + + log.info("Finished destroying connector {}", connName); + } + + // Stops a connectors tasks, then the connector + private void stopConnector(ConnectorState state) { + removeConnectorTasks(state); + try { + state.connector.stop(); + } catch (CopycatException e) { + log.error("Error shutting down connector {}: ", state.connector, e); + } + } + + private void createConnectorTasks(ConnectorState state) { + String taskClassName = state.connector.taskClass().getName(); + + log.info("Creating tasks for connector {} of type {}", state.name, taskClassName); + + List taskConfigs = state.connector.taskConfigs(state.maxTasks); + + // Generate the final configs, including framework provided settings + Map taskProps = new HashMap<>(); + for (int i = 0; i < taskConfigs.size(); i++) { + ConnectorTaskId taskId = new ConnectorTaskId(state.name, i); + Properties config = taskConfigs.get(i); + // TODO: This probably shouldn't be in the Herder. It's nice to have Copycat ensure the list of topics + // is automatically provided to tasks since it is required by the framework, but this + String subscriptionTopics = Utils.join(state.inputTopics, ","); + if (state.connector instanceof SinkConnector) { + // Make sure we don't modify the original since the connector may reuse it internally + Properties configForSink = new Properties(); + configForSink.putAll(config); + configForSink.setProperty(SinkTask.TOPICS_CONFIG, subscriptionTopics); + config = configForSink; + } + taskProps.put(taskId, config); + } + + // And initiate the tasks + for (int i = 0; i < taskConfigs.size(); i++) { + ConnectorTaskId taskId = new ConnectorTaskId(state.name, i); + Properties config = taskProps.get(taskId); + try { + worker.addTask(taskId, taskClassName, config); + // We only need to store the task IDs so we can clean up. + state.tasks.add(taskId); + } catch (Throwable e) { + log.error("Failed to add task {}: ", taskId, e); + // Swallow this so we can continue updating the rest of the tasks + // FIXME what's the proper response? Kill all the tasks? Consider this the same as a task + // that died after starting successfully. + } + } + } + + private void removeConnectorTasks(ConnectorState state) { + Iterator taskIter = state.tasks.iterator(); + while (taskIter.hasNext()) { + ConnectorTaskId taskId = taskIter.next(); + try { + worker.stopTask(taskId); + taskIter.remove(); + } catch (CopycatException e) { + log.error("Failed to stop task {}: ", taskId, e); + // Swallow this so we can continue stopping the rest of the tasks + // FIXME: Forcibly kill the task? + } + } + } + + private void updateConnectorTasks(ConnectorState state) { + removeConnectorTasks(state); + createConnectorTasks(state); + } + + /** + * Requests reconfiguration of the task. This should only be triggered by + * {@link StandaloneConnectorContext}. + * + * @param connName name of the connector that should be reconfigured + */ + public synchronized void requestTaskReconfiguration(String connName) { + ConnectorState state = connectors.get(connName); + if (state == null) { + log.error("Task that requested reconfiguration does not exist: {}", connName); + return; + } + updateConnectorTasks(state); + } + + + private static class ConnectorState { + public String name; + public Connector connector; + public int maxTasks; + public List inputTopics; + Set tasks; + + public ConnectorState(String name, Connector connector, int maxTasks, + List inputTopics) { + this.name = name; + this.connector = connector; + this.maxTasks = maxTasks; + this.inputTopics = inputTopics; + this.tasks = new HashSet<>(); + } + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/FileOffsetBackingStore.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/FileOffsetBackingStore.java new file mode 100644 index 0000000000000..f707fd6f53753 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/FileOffsetBackingStore.java @@ -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 org.apache.kafka.copycat.storage; + +import org.apache.kafka.copycat.errors.CopycatException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.*; +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; + +/** + * Implementation of OffsetBackingStore that saves data locally to a file. To ensure this behaves + * similarly to a real backing store, operations are executed asynchronously on a background thread. + */ +public class FileOffsetBackingStore extends MemoryOffsetBackingStore { + private static final Logger log = LoggerFactory.getLogger(FileOffsetBackingStore.class); + + public final static String OFFSET_STORAGE_FILE_FILENAME_CONFIG = "offset.storage.file.filename"; + private File file; + + public FileOffsetBackingStore() { + + } + + @Override + public void configure(Map props) { + super.configure(props); + String filename = (String) props.get(OFFSET_STORAGE_FILE_FILENAME_CONFIG); + file = new File(filename); + } + + @Override + public synchronized void start() { + super.start(); + log.info("Starting FileOffsetBackingStore with file {}", file); + load(); + } + + @Override + public synchronized void stop() { + super.stop(); + // Nothing to do since this doesn't maintain any outstanding connections/data + log.info("Stopped FileOffsetBackingStore"); + } + + @SuppressWarnings("unchecked") + private void load() { + try { + ObjectInputStream is = new ObjectInputStream(new FileInputStream(file)); + Object obj = is.readObject(); + if (!(obj instanceof HashMap)) + throw new CopycatException("Expected HashMap but found " + obj.getClass()); + Map raw = (Map) obj; + data = new HashMap<>(); + for (Map.Entry mapEntry : raw.entrySet()) { + ByteBuffer key = (mapEntry.getKey() != null) ? ByteBuffer.wrap(mapEntry.getKey()) : null; + ByteBuffer value = (mapEntry.getValue() != null) ? ByteBuffer.wrap(mapEntry.getValue()) : null; + data.put(key, value); + } + is.close(); + } catch (FileNotFoundException | EOFException e) { + // FileNotFoundException: Ignore, may be new. + // EOFException: Ignore, this means the file was missing or corrupt + } catch (IOException | ClassNotFoundException e) { + throw new CopycatException(e); + } + } + + protected void save() { + try { + ObjectOutputStream os = new ObjectOutputStream(new FileOutputStream(file)); + Map raw = new HashMap<>(); + for (Map.Entry mapEntry : data.entrySet()) { + byte[] key = (mapEntry.getKey() != null) ? mapEntry.getKey().array() : null; + byte[] value = (mapEntry.getValue() != null) ? mapEntry.getValue().array() : null; + raw.put(key, value); + } + os.writeObject(raw); + os.close(); + } catch (IOException e) { + throw new CopycatException(e); + } + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/KafkaOffsetBackingStore.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/KafkaOffsetBackingStore.java new file mode 100644 index 0000000000000..a74b39cfed1e1 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/KafkaOffsetBackingStore.java @@ -0,0 +1,393 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.storage; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.ConsumerWakeupException; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +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.KafkaException; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.copycat.errors.CopycatException; +import org.apache.kafka.copycat.util.Callback; +import org.apache.kafka.copycat.util.ConvertingFutureCallback; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +/** + *

            + * Implementation of OffsetBackingStore that uses a Kafka topic to store offset data. + *

            + *

            + * Internally, this implementation both produces to and consumes from a Kafka topic which stores the offsets. + * It accepts producer and consumer overrides via its configuration but forces some settings to specific values + * to ensure correct behavior (e.g. acks, auto.offset.reset). + *

            + */ +public class KafkaOffsetBackingStore implements OffsetBackingStore { + private static final Logger log = LoggerFactory.getLogger(KafkaOffsetBackingStore.class); + + public final static String OFFSET_STORAGE_TOPIC_CONFIG = "offset.storage.topic"; + + private final static long CREATE_TOPIC_TIMEOUT_MS = 30000; + + private Time time; + private Map configs; + private String topic; + private Consumer consumer; + private Producer producer; + private HashMap data; + + private Thread thread; + private boolean stopRequested; + private Queue> readLogEndOffsetCallbacks; + + public KafkaOffsetBackingStore() { + this(new SystemTime()); + } + + public KafkaOffsetBackingStore(Time time) { + this.time = time; + } + + @Override + public void configure(Map configs) { + this.configs = configs; + topic = (String) configs.get(OFFSET_STORAGE_TOPIC_CONFIG); + if (topic == null) + throw new CopycatException("Offset storage topic must be specified"); + + data = new HashMap<>(); + stopRequested = false; + readLogEndOffsetCallbacks = new ArrayDeque<>(); + } + + @Override + public void start() { + log.info("Starting KafkaOffsetBackingStore with topic " + topic); + + producer = createProducer(); + consumer = createConsumer(); + List partitions = new ArrayList<>(); + + // Until we have admin utilities we can use to check for the existence of this topic and create it if it is missing, + // we rely on topic auto-creation + List partitionInfos = null; + long started = time.milliseconds(); + while (partitionInfos == null && time.milliseconds() - started < CREATE_TOPIC_TIMEOUT_MS) { + partitionInfos = consumer.partitionsFor(topic); + Utils.sleep(Math.min(time.milliseconds() - started, 1000)); + } + if (partitionInfos == null) + throw new CopycatException("Could not look up partition metadata for offset backing store topic in" + + " allotted period. This could indicate a connectivity issue, unavailable topic partitions, or if" + + " this is your first use of the topic it may have taken too long to create."); + + for (PartitionInfo partition : partitionInfos) + partitions.add(new TopicPartition(partition.topic(), partition.partition())); + consumer.assign(partitions); + + readToLogEnd(); + + thread = new WorkThread(); + thread.start(); + + log.info("Finished reading offsets topic and starting KafkaOffsetBackingStore"); + } + + @Override + public void stop() { + log.info("Stopping KafkaOffsetBackingStore"); + + synchronized (this) { + stopRequested = true; + consumer.wakeup(); + } + + try { + thread.join(); + } catch (InterruptedException e) { + throw new CopycatException("Failed to stop KafkaOffsetBackingStore. Exiting without cleanly shutting " + + "down it's producer and consumer.", e); + } + + try { + producer.close(); + } catch (KafkaException e) { + log.error("Failed to close KafkaOffsetBackingStore producer", e); + } + + try { + consumer.close(); + } catch (KafkaException e) { + log.error("Failed to close KafkaOffsetBackingStore consumer", e); + } + } + + @Override + public Future> get(final Collection keys, + final Callback> callback) { + ConvertingFutureCallback> future = new ConvertingFutureCallback>(callback) { + @Override + public Map convert(Void result) { + Map values = new HashMap<>(); + synchronized (KafkaOffsetBackingStore.this) { + for (ByteBuffer key : keys) + values.put(key, data.get(key)); + } + return values; + } + }; + readLogEndOffsetCallbacks.add(future); + consumer.wakeup(); + return future; + } + + @Override + public Future set(final Map values, final Callback callback) { + SetCallbackFuture producerCallback = new SetCallbackFuture(values.size(), callback); + + for (Map.Entry entry : values.entrySet()) { + producer.send(new ProducerRecord<>(topic, entry.getKey().array(), entry.getValue().array()), producerCallback); + } + + return producerCallback; + } + + + + private Producer createProducer() { + Map producerProps = new HashMap<>(); + producerProps.putAll(configs); + 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.put(ProducerConfig.ACKS_CONFIG, "all"); + return new KafkaProducer<>(producerProps); + } + + private Consumer createConsumer() { + Map consumerConfig = new HashMap<>(); + consumerConfig.putAll(configs); + consumerConfig.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false); + consumerConfig.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + consumerConfig.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + consumerConfig.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + return new KafkaConsumer<>(consumerConfig); + } + + private void poll(long timeoutMs) { + try { + ConsumerRecords records = consumer.poll(timeoutMs); + for (ConsumerRecord record : records) { + ByteBuffer key = record.key() != null ? ByteBuffer.wrap((byte[]) record.key()) : null; + ByteBuffer value = record.value() != null ? ByteBuffer.wrap((byte[]) record.value()) : null; + data.put(key, value); + } + } catch (ConsumerWakeupException e) { + // Expected on get() or stop(). The calling code should handle this + throw e; + } catch (KafkaException e) { + log.error("Error polling: " + e); + } + } + + private void readToLogEnd() { + log.trace("Reading to end of offset log"); + + Set assignment = consumer.assignment(); + + // This approach to getting the current end offset is hacky until we have an API for looking these up directly + Map offsets = new HashMap<>(); + for (TopicPartition tp : assignment) { + long offset = consumer.position(tp); + offsets.put(tp, offset); + consumer.seekToEnd(tp); + } + + Map endOffsets = new HashMap<>(); + try { + poll(0); + } finally { + // If there is an exception, even a possibly expected one like ConsumerWakeupException, we need to make sure + // the consumers position is reset or it'll get into an inconsistent state. + for (TopicPartition tp : assignment) { + long startOffset = offsets.get(tp); + long endOffset = consumer.position(tp); + if (endOffset > startOffset) { + endOffsets.put(tp, endOffset); + consumer.seek(tp, startOffset); + } + log.trace("Reading to end of log for {}: starting offset {} to ending offset {}", tp, startOffset, endOffset); + } + } + + while (!endOffsets.isEmpty()) { + poll(Integer.MAX_VALUE); + + Iterator> it = endOffsets.entrySet().iterator(); + while (it.hasNext()) { + Map.Entry entry = it.next(); + if (consumer.position(entry.getKey()) >= entry.getValue()) + it.remove(); + else + break; + } + } + } + + + private static class SetCallbackFuture implements org.apache.kafka.clients.producer.Callback, Future { + private int numLeft; + private boolean completed = false; + private Throwable exception = null; + private final Callback callback; + + public SetCallbackFuture(int numRecords, Callback callback) { + numLeft = numRecords; + this.callback = callback; + } + + @Override + public synchronized void onCompletion(RecordMetadata metadata, Exception exception) { + if (exception != null) { + if (!completed) { + this.exception = exception; + callback.onCompletion(exception, null); + completed = true; + this.notify(); + } + return; + } + + numLeft -= 1; + if (numLeft == 0) { + callback.onCompletion(null, null); + completed = true; + this.notify(); + } + } + + @Override + public synchronized boolean cancel(boolean mayInterruptIfRunning) { + return false; + } + + @Override + public synchronized boolean isCancelled() { + return false; + } + + @Override + public synchronized boolean isDone() { + return completed; + } + + @Override + public synchronized Void get() throws InterruptedException, ExecutionException { + while (!completed) { + this.wait(); + } + if (exception != null) + throw new ExecutionException(exception); + return null; + } + + @Override + public synchronized Void get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { + long started = System.currentTimeMillis(); + long limit = started + unit.toMillis(timeout); + while (!completed) { + long leftMs = limit - System.currentTimeMillis(); + if (leftMs < 0) + throw new TimeoutException("KafkaOffsetBackingStore Future timed out."); + this.wait(leftMs); + } + if (exception != null) + throw new ExecutionException(exception); + return null; + } + } + + private class WorkThread extends Thread { + @Override + public void run() { + try { + while (true) { + int numCallbacks; + synchronized (KafkaOffsetBackingStore.this) { + if (stopRequested) + break; + numCallbacks = readLogEndOffsetCallbacks.size(); + } + + if (numCallbacks > 0) { + try { + readToLogEnd(); + } catch (ConsumerWakeupException e) { + // Either received another get() call and need to retry reading to end of log or stop() was + // called. Both are handled by restarting this loop. + continue; + } + } + + synchronized (KafkaOffsetBackingStore.this) { + for (int i = 0; i < numCallbacks; i++) { + Callback cb = readLogEndOffsetCallbacks.poll(); + cb.onCompletion(null, null); + } + } + + try { + poll(Integer.MAX_VALUE); + } catch (ConsumerWakeupException e) { + // See previous comment, both possible causes of this wakeup are handled by starting this loop again + continue; + } + } + } catch (Throwable t) { + log.error("Unexpected exception in KafkaOffsetBackingStore's work thread", t); + } + } + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/MemoryOffsetBackingStore.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/MemoryOffsetBackingStore.java new file mode 100644 index 0000000000000..11a1b891ffd0e --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/MemoryOffsetBackingStore.java @@ -0,0 +1,105 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

            + * http://www.apache.org/licenses/LICENSE-2.0 + *

            + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.storage; + +import org.apache.kafka.copycat.util.Callback; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +/** + * Implementation of OffsetBackingStore that doesn't actually persist any data. To ensure this + * behaves similarly to a real backing store, operations are executed asynchronously on a + * background thread. + */ +public class MemoryOffsetBackingStore implements OffsetBackingStore { + private static final Logger log = LoggerFactory.getLogger(MemoryOffsetBackingStore.class); + + protected Map data = new HashMap<>(); + protected ExecutorService executor = Executors.newSingleThreadExecutor(); + + public MemoryOffsetBackingStore() { + + } + + @Override + public void configure(Map props) { + } + + @Override + public synchronized void start() { + } + + @Override + public synchronized void stop() { + // Nothing to do since this doesn't maintain any outstanding connections/data + } + + @Override + public Future> get( + final Collection keys, + final Callback> callback) { + return executor.submit(new Callable>() { + @Override + public Map call() throws Exception { + Map result = new HashMap<>(); + synchronized (MemoryOffsetBackingStore.this) { + for (ByteBuffer key : keys) { + result.put(key, data.get(key)); + } + } + if (callback != null) + callback.onCompletion(null, result); + return result; + } + }); + + } + + @Override + public Future set(final Map values, + final Callback callback) { + return executor.submit(new Callable() { + @Override + public Void call() throws Exception { + synchronized (MemoryOffsetBackingStore.this) { + for (Map.Entry entry : values.entrySet()) { + data.put(entry.getKey(), entry.getValue()); + } + save(); + } + if (callback != null) + callback.onCompletion(null, null); + return null; + } + }); + } + + // Hook to allow subclasses to persist data + protected void save() { + + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/OffsetBackingStore.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/OffsetBackingStore.java new file mode 100644 index 0000000000000..239d9a8d73b76 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/OffsetBackingStore.java @@ -0,0 +1,72 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

            + * http://www.apache.org/licenses/LICENSE-2.0 + *

            + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.storage; + +import org.apache.kafka.common.Configurable; +import org.apache.kafka.copycat.util.Callback; + +import java.nio.ByteBuffer; +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.Future; + +/** + *

            + * OffsetBackingStore is an interface for storage backends that store key-value data. The backing + * store doesn't need to handle serialization or deserialization. It only needs to support + * reading/writing bytes. Since it is expected these operations will require network + * operations, only bulk operations are supported. + *

            + *

            + * Since OffsetBackingStore is a shared resource that may be used by many OffsetStorage instances + * that are associated with individual tasks, the caller must be sure keys include information about the + * connector so that the shared namespace does not result in conflicting keys. + *

            + */ +public interface OffsetBackingStore extends Configurable { + + /** + * Start this offset store. + */ + public void start(); + + /** + * Stop the backing store. Implementations should attempt to shutdown gracefully, but not block + * indefinitely. + */ + public void stop(); + + /** + * Get the values for the specified keys + * @param keys list of keys to look up + * @param callback callback to invoke on completion + * @return future for the resulting map from key to value + */ + public Future> get( + Collection keys, + Callback> callback); + + /** + * Set the specified keys and values. + * @param values map from key to value + * @param callback callback to invoke on completion + * @return void future for the operation + */ + public Future set(Map values, + Callback callback); +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageReaderImpl.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageReaderImpl.java new file mode 100644 index 0000000000000..dbb3d0d087304 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageReaderImpl.java @@ -0,0 +1,110 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

            + * http://www.apache.org/licenses/LICENSE-2.0 + *

            + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.storage; + +import org.apache.kafka.copycat.data.SchemaAndValue; +import org.apache.kafka.copycat.errors.CopycatException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; + +/** + * Implementation of OffsetStorageReader. Unlike OffsetStorageWriter which is implemented + * directly, the interface is only separate from this implementation because it needs to be + * included in the public API package. + */ +public class OffsetStorageReaderImpl implements OffsetStorageReader { + private static final Logger log = LoggerFactory.getLogger(OffsetStorageReaderImpl.class); + + private final OffsetBackingStore backingStore; + private final String namespace; + private final Converter keyConverter; + private final Converter valueConverter; + + public OffsetStorageReaderImpl(OffsetBackingStore backingStore, String namespace, + Converter keyConverter, Converter valueConverter) { + this.backingStore = backingStore; + this.namespace = namespace; + this.keyConverter = keyConverter; + this.valueConverter = valueConverter; + } + + @Override + public Map offset(Map partition) { + return offsets(Arrays.asList(partition)).get(partition); + } + + @Override + public Map, Map> offsets(Collection> partitions) { + // Serialize keys so backing store can work with them + Map> serializedToOriginal = new HashMap<>(partitions.size()); + for (Map key : partitions) { + try { + // Offsets are treated as schemaless, their format is only validated here (and the returned value below) + OffsetUtils.validateFormat(key); + byte[] keySerialized = keyConverter.fromCopycatData(namespace, null, Arrays.asList(namespace, key)); + ByteBuffer keyBuffer = (keySerialized != null) ? ByteBuffer.wrap(keySerialized) : null; + serializedToOriginal.put(keyBuffer, key); + } catch (Throwable t) { + log.error("CRITICAL: Failed to serialize partition key when getting offsets for task with " + + "namespace {}. No value for this data will be returned, which may break the " + + "task or cause it to skip some data.", namespace, t); + } + } + + // Get serialized key -> serialized value from backing store + Map raw; + try { + raw = backingStore.get(serializedToOriginal.keySet(), null).get(); + } catch (Exception e) { + log.error("Failed to fetch offsets from namespace {}: ", namespace, e); + throw new CopycatException("Failed to fetch offsets.", e); + } + + // Deserialize all the values and map back to the original keys + Map, Map> result = new HashMap<>(partitions.size()); + for (Map.Entry rawEntry : raw.entrySet()) { + try { + // Since null could be a valid key, explicitly check whether map contains the key + if (!serializedToOriginal.containsKey(rawEntry.getKey())) { + log.error("Should be able to map {} back to a requested partition-offset key, backing " + + "store may have returned invalid data", rawEntry.getKey()); + continue; + } + Map origKey = serializedToOriginal.get(rawEntry.getKey()); + SchemaAndValue deserializedSchemaAndValue = valueConverter.toCopycatData(namespace, rawEntry.getValue().array()); + Object deserializedValue = deserializedSchemaAndValue.value(); + OffsetUtils.validateFormat(deserializedValue); + + result.put(origKey, (Map) deserializedValue); + } catch (Throwable t) { + log.error("CRITICAL: Failed to deserialize offset data when getting offsets for task with" + + " namespace {}. No value for this data will be returned, which may break the " + + "task or cause it to skip some data. This could either be due to an error in " + + "the connector implementation or incompatible schema.", namespace, t); + } + } + + return result; + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageWriter.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageWriter.java new file mode 100644 index 0000000000000..59c12a7c9ade3 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageWriter.java @@ -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 org.apache.kafka.copycat.storage; + +import org.apache.kafka.copycat.errors.CopycatException; +import org.apache.kafka.copycat.util.Callback; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.util.*; +import java.util.concurrent.Future; + +/** + *

            + * OffsetStorageWriter is a buffered writer that wraps the simple OffsetBackingStore interface. + * It maintains a copy of the key-value data in memory and buffers writes. It allows you to take + * a snapshot, which can then be asynchronously flushed to the backing store while new writes + * continue to be processed. This allows Copycat to process offset commits in the background + * while continuing to process messages. + *

            + *

            + * Copycat uses an OffsetStorage implementation to save state about the current progress of + * source (import to Kafka) jobs, which may have many input partitions and "offsets" may not be as + * simple as they are for Kafka partitions or files. Offset storage is not required for sink jobs + * because they can use Kafka's native offset storage (or the sink data store can handle offset + * storage to achieve exactly once semantics). + *

            + *

            + * Both partitions and offsets are generic data objects. This allows different connectors to use + * whatever representation they need, even arbitrarily complex records. These are translated + * internally into the serialized form the OffsetBackingStore uses. + *

            + *

            + * Note that this only provides write functionality. This is intentional to ensure stale data is + * never read. Offset data should only be read during startup or reconfiguration of a task. By + * always serving those requests by reading the values from the backing store, we ensure we never + * accidentally use stale data. (One example of how this can occur: a task is processing input + * partition A, writing offsets; reconfiguration causes partition A to be reassigned elsewhere; + * reconfiguration causes partition A to be reassigned to this node, but now the offset data is out + * of date). Since these offsets are created and managed by the connector itself, there's no way + * for the offset management layer to know which keys are "owned" by which tasks at any given + * time. + *

            + *

            + * This class is not thread-safe. It should only be accessed from a Task's processing thread. + *

            + */ +public class OffsetStorageWriter { + private static final Logger log = LoggerFactory.getLogger(OffsetStorageWriter.class); + + private final OffsetBackingStore backingStore; + private final Converter keyConverter; + private final Converter valueConverter; + private final String namespace; + // Offset data in Copycat format + private Map, Map> data = new HashMap<>(); + + // Not synchronized, should only be accessed by flush thread + private Map, Map> toFlush = null; + // Unique ID for each flush request to handle callbacks after timeouts + private long currentFlushId = 0; + + public OffsetStorageWriter(OffsetBackingStore backingStore, + String namespace, Converter keyConverter, Converter valueConverter) { + this.backingStore = backingStore; + this.namespace = namespace; + this.keyConverter = keyConverter; + this.valueConverter = valueConverter; + } + + /** + * Set an offset for a partition using Copycat data values + * @param partition the partition to store an offset for + * @param offset the offset + */ + public synchronized void offset(Map partition, Map offset) { + data.put((Map) partition, (Map) offset); + } + + private boolean flushing() { + return toFlush != null; + } + + /** + * Performs the first step of a flush operation, snapshotting the current state. This does not + * actually initiate the flush with the underlying storage. + * + * @return true if a flush was initiated, false if no data was available + */ + public synchronized boolean beginFlush() { + if (flushing()) { + log.error("Invalid call to OffsetStorageWriter flush() while already flushing, the " + + "framework should not allow this"); + throw new CopycatException("OffsetStorageWriter is already flushing"); + } + + if (data.isEmpty()) + return false; + + assert !flushing(); + toFlush = data; + data = new HashMap<>(); + return true; + } + + /** + * Flush the current offsets and clear them from this writer. This is non-blocking: it + * moves the current set of offsets out of the way, serializes the data, and asynchronously + * writes the data to the backing store. If no offsets need to be written, the callback is + * still invoked, but no Future is returned. + * + * @return a Future, or null if there are no offsets to commitOffsets + */ + public Future doFlush(final Callback callback) { + final long flushId = currentFlushId; + + // Serialize + Map offsetsSerialized; + try { + offsetsSerialized = new HashMap<>(); + for (Map.Entry, Map> entry : toFlush.entrySet()) { + // Offsets are specified as schemaless to the converter, using whatever internal schema is appropriate + // for that data. The only enforcement of the format is here. + OffsetUtils.validateFormat(entry.getKey()); + OffsetUtils.validateFormat(entry.getValue()); + // When serializing the key, we add in the namespace information so the key is [namespace, real key] + byte[] key = keyConverter.fromCopycatData(namespace, null, Arrays.asList(namespace, entry.getKey())); + ByteBuffer keyBuffer = (key != null) ? ByteBuffer.wrap(key) : null; + byte[] value = valueConverter.fromCopycatData(namespace, null, entry.getValue()); + ByteBuffer valueBuffer = (value != null) ? ByteBuffer.wrap(value) : null; + offsetsSerialized.put(keyBuffer, valueBuffer); + } + } catch (Throwable t) { + // Must handle errors properly here or the writer will be left mid-flush forever and be + // unable to make progress. + log.error("CRITICAL: Failed to serialize offset data, making it impossible to commit " + + "offsets under namespace {}. This likely won't recover unless the " + + "unserializable partition or offset information is overwritten.", namespace); + log.error("Cause of serialization failure:", t); + callback.onCompletion(t, null); + return null; + } + + // And submit the data + log.debug("Submitting {} entries to backing store", offsetsSerialized.size()); + return backingStore.set(offsetsSerialized, new Callback() { + @Override + public void onCompletion(Throwable error, Void result) { + boolean isCurrent = handleFinishWrite(flushId, error, result); + if (isCurrent && callback != null) + callback.onCompletion(error, result); + } + }); + } + + /** + * Cancel a flush that has been initiated by {@link #beginFlush}. This should not be called if + * {@link #doFlush} has already been invoked. It should be used if an operation performed + * between beginFlush and doFlush failed. + */ + public synchronized void cancelFlush() { + // Verify we're still flushing data to handle a race between cancelFlush() calls from up the + // call stack and callbacks from the write request to underlying storage + if (flushing()) { + // Just recombine the data and place it back in the primary storage + toFlush.putAll(data); + data = toFlush; + currentFlushId++; + toFlush = null; + } + } + + /** + * Handle completion of a write. Returns true if this callback is for the current flush + * operation, false if it's for an old operation that should now be ignored. + */ + private synchronized boolean handleFinishWrite(long flushId, Throwable error, Void result) { + // Callbacks need to be handled carefully since the flush operation may have already timed + // out and been cancelled. + if (flushId != currentFlushId) + return false; + + if (error != null) { + cancelFlush(); + } else { + currentFlushId++; + toFlush = null; + } + return true; + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/OffsetUtils.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/OffsetUtils.java new file mode 100644 index 0000000000000..bd3a87b5b230a --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/OffsetUtils.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.copycat.storage; + +import org.apache.kafka.copycat.data.CopycatSchema; +import org.apache.kafka.copycat.data.Schema; +import org.apache.kafka.copycat.errors.DataException; + +import java.util.Map; + +public class OffsetUtils { + public static void validateFormat(Object offsetData) { + if (!(offsetData instanceof Map)) + throw new DataException("Offsets must be specified as a Map"); + validateFormat((Map) offsetData); + } + + public static void validateFormat(Map offsetData) { + for (Map.Entry entry : offsetData.entrySet()) { + if (!(entry.getKey() instanceof String)) + throw new DataException("Offsets may only use String keys"); + + Object value = entry.getValue(); + if (value == null) + continue; + Schema.Type schemaType = CopycatSchema.schemaType(value.getClass()); + if (!schemaType.isPrimitive()) + throw new DataException("Offsets may only contain primitive types as values, but field " + entry.getKey() + " contains " + schemaType); + } + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/Callback.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/Callback.java new file mode 100644 index 0000000000000..5cf1423ccc182 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/Callback.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

            + * http://www.apache.org/licenses/LICENSE-2.0 + *

            + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.util; + +/** + * Generic interface for callbacks + */ +public interface Callback { + /** + * Invoked upon completion of the operation. + * + * @param error the error that caused the operation to fail, or null if no error occurred + * @param result the return value, or null if the operation failed + */ + void onCompletion(Throwable error, V result); +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/ConnectorTaskId.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/ConnectorTaskId.java new file mode 100644 index 0000000000000..683c634ced11a --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/ConnectorTaskId.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.copycat.util; + +import java.io.Serializable; + +/** + * Unique ID for a single task. It includes a unique connector ID and a task ID that is unique within + * the connector. + */ +public class ConnectorTaskId implements Serializable { + private final String connector; + private final int task; + + public ConnectorTaskId(String job, int task) { + this.connector = job; + this.task = task; + } + + public String connector() { + return connector; + } + + public int task() { + return task; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + ConnectorTaskId that = (ConnectorTaskId) o; + + if (task != that.task) + return false; + if (connector != null ? !connector.equals(that.connector) : that.connector != null) + return false; + + return true; + } + + @Override + public int hashCode() { + int result = connector != null ? connector.hashCode() : 0; + result = 31 * result + task; + return result; + } + + @Override + public String toString() { + return connector + '-' + task; + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/ConvertingFutureCallback.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/ConvertingFutureCallback.java new file mode 100644 index 0000000000000..6bf38857933e5 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/ConvertingFutureCallback.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.copycat.util; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +public abstract class ConvertingFutureCallback implements Callback, Future { + + private Callback underlying; + private CountDownLatch finishedLatch; + private T result = null; + private Throwable exception = null; + + public ConvertingFutureCallback(Callback underlying) { + this.underlying = underlying; + this.finishedLatch = new CountDownLatch(1); + } + + public abstract T convert(U result); + + @Override + public void onCompletion(Throwable error, U result) { + this.exception = error; + this.result = convert(result); + if (underlying != null) + underlying.onCompletion(error, this.result); + finishedLatch.countDown(); + } + + @Override + public boolean cancel(boolean b) { + return false; + } + + @Override + public boolean isCancelled() { + return false; + } + + @Override + public boolean isDone() { + return finishedLatch.getCount() == 0; + } + + @Override + public T get() throws InterruptedException, ExecutionException { + finishedLatch.await(); + return result(); + } + + @Override + public T get(long l, TimeUnit timeUnit) + throws InterruptedException, ExecutionException, TimeoutException { + finishedLatch.await(l, timeUnit); + return result(); + } + + private T result() throws ExecutionException { + if (exception != null) { + throw new ExecutionException(exception); + } + return result; + } +} + diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/FutureCallback.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/FutureCallback.java new file mode 100644 index 0000000000000..61e04b6d53441 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/FutureCallback.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.copycat.util; + +public class FutureCallback extends ConvertingFutureCallback { + + public FutureCallback(Callback underlying) { + super(underlying); + } + + @Override + public T convert(T result) { + return result; + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/ShutdownableThread.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/ShutdownableThread.java new file mode 100644 index 0000000000000..3e23f29fed4ff --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/ShutdownableThread.java @@ -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 org.apache.kafka.copycat.util; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + *

            + * Thread class with support for triggering graceful and forcible shutdown. In graceful shutdown, + * a flag is set, which the thread should detect and try to exit gracefully from. In forcible + * shutdown, the thread is interrupted. These can be combined to give a thread a chance to exit + * gracefully, but then force it to exit if it takes too long. + *

            + *

            + * Implementations should override the {@link #execute} method and check {@link #getRunning} to + * determine whether they should try to gracefully exit. + *

            + */ +public abstract class ShutdownableThread extends Thread { + private static final Logger log = LoggerFactory.getLogger(ShutdownableThread.class); + + private AtomicBoolean isRunning = new AtomicBoolean(true); + private CountDownLatch shutdownLatch = new CountDownLatch(1); + + /** + * An UncaughtExceptionHandler to register on every instance of this class. This is useful for + * testing, where AssertionExceptions in the thread may not cause the test to fail. Since one + * instance is used for all threads, it must be thread-safe. + */ + volatile public static UncaughtExceptionHandler funcaughtExceptionHandler = null; + + public ShutdownableThread(String name) { + // The default is daemon=true so that these threads will not prevent shutdown. We use this + // default because threads that are running user code that may not clean up properly, even + // when we attempt to forcibly shut them down. + this(name, true); + } + + public ShutdownableThread(String name, boolean daemon) { + super(name); + this.setDaemon(daemon); + if (funcaughtExceptionHandler != null) + this.setUncaughtExceptionHandler(funcaughtExceptionHandler); + } + + /** + * Implementations should override this method with the main body for the thread. + */ + public abstract void execute(); + + /** + * Returns true if the thread hasn't exited yet and none of the shutdown methods have been + * invoked + */ + public boolean getRunning() { + return isRunning.get(); + } + + @Override + public void run() { + try { + execute(); + } catch (Error | RuntimeException e) { + log.error("Thread {} exiting with uncaught exception: ", getName(), e); + throw e; + } finally { + shutdownLatch.countDown(); + } + } + + /** + * Shutdown the thread, first trying to shut down gracefully using the specified timeout, then + * forcibly interrupting the thread. + * @param gracefulTimeout the maximum time to wait for a graceful exit + * @param unit the time unit of the timeout argument + */ + public void shutdown(long gracefulTimeout, TimeUnit unit) + throws InterruptedException { + boolean success = gracefulShutdown(gracefulTimeout, unit); + if (!success) + forceShutdown(); + } + + /** + * Attempt graceful shutdown + * @param timeout the maximum time to wait + * @param unit the time unit of the timeout argument + * @return true if successful, false if the timeout elapsed + */ + public boolean gracefulShutdown(long timeout, TimeUnit unit) throws InterruptedException { + startGracefulShutdown(); + return awaitShutdown(timeout, unit); + } + + /** + * Start shutting down this thread gracefully, but do not block waiting for it to exit. + */ + public void startGracefulShutdown() { + log.info("Starting graceful shutdown of thread {}", getName()); + isRunning.set(false); + } + + /** + * Awaits shutdown of this thread, waiting up to the timeout. + * @param timeout the maximum time to wait + * @param unit the time unit of the timeout argument + * @return true if successful, false if the timeout elapsed + * @throws InterruptedException + */ + public boolean awaitShutdown(long timeout, TimeUnit unit) throws InterruptedException { + return shutdownLatch.await(timeout, unit); + } + + /** + * Immediately tries to force the thread to shut down by interrupting it. This does not try to + * wait for the thread to truly exit because forcible shutdown is not always possible. By + * default, threads are marked as daemon threads so they will not prevent the process from + * exiting. + */ + public void forceShutdown() throws InterruptedException { + log.info("Forcing shutdown of thread {}", getName()); + isRunning.set(false); + interrupt(); + } +} diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/WorkerSinkTaskTest.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/WorkerSinkTaskTest.java new file mode 100644 index 0000000000000..e4d1d8ecb6a3d --- /dev/null +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/WorkerSinkTaskTest.java @@ -0,0 +1,480 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

            + * http://www.apache.org/licenses/LICENSE-2.0 + *

            + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.runtime; + +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.clients.consumer.*; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.copycat.cli.WorkerConfig; +import org.apache.kafka.copycat.data.Schema; +import org.apache.kafka.copycat.data.SchemaAndValue; +import org.apache.kafka.copycat.errors.CopycatException; +import org.apache.kafka.copycat.sink.SinkRecord; +import org.apache.kafka.copycat.sink.SinkTask; +import org.apache.kafka.copycat.sink.SinkTaskContext; +import org.apache.kafka.copycat.storage.Converter; +import org.apache.kafka.copycat.util.ConnectorTaskId; +import org.apache.kafka.copycat.util.MockTime; +import org.apache.kafka.copycat.util.ThreadedTest; +import org.easymock.*; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.api.easymock.annotation.Mock; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; +import org.powermock.reflect.Whitebox; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Properties; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +@RunWith(PowerMockRunner.class) +@PrepareForTest(WorkerSinkTask.class) +@PowerMockIgnore("javax.management.*") +public class WorkerSinkTaskTest extends ThreadedTest { + + // These are fixed to keep this code simpler. In this example we assume byte[] raw values + // with mix of integer/string in Copycat + private static final String TOPIC = "test"; + private static final int PARTITION = 12; + private static final int PARTITION2 = 13; + private static final int PARTITION3 = 14; + private static final long FIRST_OFFSET = 45; + private static final Schema KEY_SCHEMA = Schema.INT32_SCHEMA; + private static final int KEY = 12; + private static final Schema VALUE_SCHEMA = Schema.STRING_SCHEMA; + private static final String VALUE = "VALUE"; + private static final byte[] RAW_KEY = "key".getBytes(); + private static final byte[] RAW_VALUE = "value".getBytes(); + + private static final TopicPartition TOPIC_PARTITION = new TopicPartition(TOPIC, PARTITION); + private static final TopicPartition TOPIC_PARTITION2 = new TopicPartition(TOPIC, PARTITION2); + private static final TopicPartition TOPIC_PARTITION3 = new TopicPartition(TOPIC, PARTITION3); + private static final TopicPartition UNASSIGNED_TOPIC_PARTITION = new TopicPartition(TOPIC, 200); + + private ConnectorTaskId taskId = new ConnectorTaskId("job", 0); + private Time time; + @Mock private SinkTask sinkTask; + private Capture sinkTaskContext = EasyMock.newCapture(); + private WorkerConfig workerConfig; + @Mock private Converter keyConverter; + @Mock + private Converter valueConverter; + private WorkerSinkTask workerTask; + @Mock private KafkaConsumer consumer; + private WorkerSinkTaskThread workerThread; + + private long recordsReturned; + + @SuppressWarnings("unchecked") + @Override + public void setup() { + super.setup(); + time = new MockTime(); + Properties workerProps = new Properties(); + workerProps.setProperty("key.converter", "org.apache.kafka.copycat.json.JsonConverter"); + workerProps.setProperty("value.converter", "org.apache.kafka.copycat.json.JsonConverter"); + workerProps.setProperty("offset.key.converter", "org.apache.kafka.copycat.json.JsonConverter"); + workerProps.setProperty("offset.value.converter", "org.apache.kafka.copycat.json.JsonConverter"); + workerProps.setProperty("offset.key.converter.schemas.enable", "false"); + workerProps.setProperty("offset.value.converter.schemas.enable", "false"); + workerConfig = new WorkerConfig(workerProps); + workerTask = PowerMock.createPartialMock( + WorkerSinkTask.class, new String[]{"createConsumer", "createWorkerThread"}, + taskId, sinkTask, workerConfig, keyConverter, valueConverter, time); + + recordsReturned = 0; + } + + @Test + public void testPollsInBackground() throws Exception { + Properties taskProps = new Properties(); + + expectInitializeTask(taskProps); + Capture> capturedRecords = expectPolls(1L); + expectStopTask(10L); + + PowerMock.replayAll(); + + workerTask.start(taskProps); + for (int i = 0; i < 10; i++) { + workerThread.iteration(); + } + workerTask.stop(); + // No need for awaitStop since the thread is mocked + workerTask.close(); + + // Verify contents match expected values, i.e. that they were translated properly. With max + // batch size 1 and poll returns 1 message at a time, we should have a matching # of batches + assertEquals(10, capturedRecords.getValues().size()); + int offset = 0; + for (Collection recs : capturedRecords.getValues()) { + assertEquals(1, recs.size()); + for (SinkRecord rec : recs) { + SinkRecord referenceSinkRecord + = new SinkRecord(TOPIC, PARTITION, KEY_SCHEMA, KEY, VALUE_SCHEMA, VALUE, FIRST_OFFSET + offset); + assertEquals(referenceSinkRecord, rec); + offset++; + } + } + + PowerMock.verifyAll(); + } + + @Test + public void testDeliverConvertsData() throws Exception { + // Validate conversion is performed when data is delivered + SchemaAndValue record = new SchemaAndValue(Schema.INT32_SCHEMA, 12); + + ConsumerRecords records = new ConsumerRecords<>( + Collections.singletonMap( + new TopicPartition(TOPIC, 0), + Collections.singletonList(new ConsumerRecord<>(TOPIC, 0, 0, RAW_KEY, RAW_VALUE)))); + + // Exact data doesn't matter, but should be passed directly to sink task + EasyMock.expect(keyConverter.toCopycatData(EasyMock.eq(TOPIC), EasyMock.aryEq(RAW_KEY))).andReturn(record); + EasyMock.expect(valueConverter.toCopycatData(EasyMock.eq(TOPIC), EasyMock.aryEq(RAW_VALUE))).andReturn(record); + Capture> capturedRecords + = EasyMock.newCapture(CaptureType.ALL); + sinkTask.put(EasyMock.capture(capturedRecords)); + EasyMock.expectLastCall(); + + PowerMock.replayAll(); + + Whitebox.invokeMethod(workerTask, "deliverMessages", records); + assertEquals(record.schema(), capturedRecords.getValue().iterator().next().keySchema()); + assertEquals(record.value(), capturedRecords.getValue().iterator().next().key()); + assertEquals(record.schema(), capturedRecords.getValue().iterator().next().valueSchema()); + assertEquals(record.value(), capturedRecords.getValue().iterator().next().value()); + + PowerMock.verifyAll(); + } + + @Test + public void testCommit() throws Exception { + Properties taskProps = new Properties(); + + expectInitializeTask(taskProps); + // Make each poll() take the offset commit interval + Capture> capturedRecords + = expectPolls(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_DEFAULT); + expectOffsetFlush(1L, null, null, 0, true); + expectStopTask(2); + + PowerMock.replayAll(); + + workerTask.start(taskProps); + // First iteration gets one record + workerThread.iteration(); + // Second triggers commit, gets a second offset + workerThread.iteration(); + // Commit finishes synchronously for testing so we can check this immediately + assertEquals(0, workerThread.commitFailures()); + workerTask.stop(); + workerTask.close(); + + assertEquals(2, capturedRecords.getValues().size()); + + PowerMock.verifyAll(); + } + + @Test + public void testCommitTaskFlushFailure() throws Exception { + Properties taskProps = new Properties(); + + expectInitializeTask(taskProps); + Capture> capturedRecords + = expectPolls(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_DEFAULT); + expectOffsetFlush(1L, new RuntimeException(), null, 0, true); + expectStopTask(2); + + PowerMock.replayAll(); + + workerTask.start(taskProps); + // Second iteration triggers commit + workerThread.iteration(); + workerThread.iteration(); + assertEquals(1, workerThread.commitFailures()); + assertEquals(false, Whitebox.getInternalState(workerThread, "committing")); + workerTask.stop(); + workerTask.close(); + + PowerMock.verifyAll(); + } + + @Test + public void testCommitConsumerFailure() throws Exception { + Properties taskProps = new Properties(); + + expectInitializeTask(taskProps); + Capture> capturedRecords + = expectPolls(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_DEFAULT); + expectOffsetFlush(1L, null, new Exception(), 0, true); + expectStopTask(2); + + PowerMock.replayAll(); + + workerTask.start(taskProps); + // Second iteration triggers commit + workerThread.iteration(); + workerThread.iteration(); + // TODO Response to consistent failures? + assertEquals(1, workerThread.commitFailures()); + assertEquals(false, Whitebox.getInternalState(workerThread, "committing")); + workerTask.stop(); + workerTask.close(); + + PowerMock.verifyAll(); + } + + @Test + public void testCommitTimeout() throws Exception { + Properties taskProps = new Properties(); + + expectInitializeTask(taskProps); + // Cut down amount of time to pass in each poll so we trigger exactly 1 offset commit + Capture> capturedRecords + = expectPolls(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_DEFAULT / 2); + expectOffsetFlush(2L, null, null, WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_DEFAULT, false); + expectStopTask(4); + + PowerMock.replayAll(); + + workerTask.start(taskProps); + // Third iteration triggers commit, fourth gives a chance to trigger the timeout but doesn't + // trigger another commit + workerThread.iteration(); + workerThread.iteration(); + workerThread.iteration(); + workerThread.iteration(); + // TODO Response to consistent failures? + assertEquals(1, workerThread.commitFailures()); + assertEquals(false, Whitebox.getInternalState(workerThread, "committing")); + workerTask.stop(); + workerTask.close(); + + PowerMock.verifyAll(); + } + + @Test + public void testAssignmentPauseResume() throws Exception { + // Just validate that the calls are passed through to the consumer, and that where appropriate errors are + // converted + + Properties taskProps = new Properties(); + + expectInitializeTask(taskProps); + + expectOnePoll().andAnswer(new IAnswer() { + @Override + public Object answer() throws Throwable { + assertEquals(new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3)), + sinkTaskContext.getValue().assignment()); + return null; + } + }); + EasyMock.expect(consumer.assignment()).andReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3))); + + expectOnePoll().andAnswer(new IAnswer() { + @Override + public Object answer() throws Throwable { + try { + sinkTaskContext.getValue().pause(UNASSIGNED_TOPIC_PARTITION); + fail("Trying to pause unassigned partition should have thrown an Copycat exception"); + } catch (CopycatException e) { + // expected + } + sinkTaskContext.getValue().pause(TOPIC_PARTITION, TOPIC_PARTITION2); + return null; + } + }); + consumer.pause(UNASSIGNED_TOPIC_PARTITION); + PowerMock.expectLastCall().andThrow(new IllegalStateException("unassigned topic partition")); + consumer.pause(TOPIC_PARTITION, TOPIC_PARTITION2); + PowerMock.expectLastCall(); + + expectOnePoll().andAnswer(new IAnswer() { + @Override + public Object answer() throws Throwable { + try { + sinkTaskContext.getValue().resume(UNASSIGNED_TOPIC_PARTITION); + fail("Trying to resume unassigned partition should have thrown an Copycat exception"); + } catch (CopycatException e) { + // expected + } + + sinkTaskContext.getValue().resume(TOPIC_PARTITION, TOPIC_PARTITION2); + return null; + } + }); + consumer.resume(UNASSIGNED_TOPIC_PARTITION); + PowerMock.expectLastCall().andThrow(new IllegalStateException("unassigned topic partition")); + consumer.resume(TOPIC_PARTITION, TOPIC_PARTITION2); + PowerMock.expectLastCall(); + + expectStopTask(0); + + PowerMock.replayAll(); + + workerTask.start(taskProps); + workerThread.iteration(); + workerThread.iteration(); + workerThread.iteration(); + workerTask.stop(); + workerTask.close(); + + PowerMock.verifyAll(); + } + + + private void expectInitializeTask(Properties taskProps) throws Exception { + sinkTask.initialize(EasyMock.capture(sinkTaskContext)); + PowerMock.expectLastCall(); + sinkTask.start(taskProps); + PowerMock.expectLastCall(); + + PowerMock.expectPrivate(workerTask, "createConsumer", taskProps) + .andReturn(consumer); + workerThread = PowerMock.createPartialMock(WorkerSinkTaskThread.class, new String[]{"start"}, + workerTask, "mock-worker-thread", time, + workerConfig); + PowerMock.expectPrivate(workerTask, "createWorkerThread") + .andReturn(workerThread); + workerThread.start(); + PowerMock.expectLastCall(); + } + + private void expectStopTask(final long expectedMessages) throws Exception { + final long finalOffset = FIRST_OFFSET + expectedMessages - 1; + + sinkTask.stop(); + PowerMock.expectLastCall(); + + // No offset commit since it happens in the mocked worker thread, but the main thread does need to wake up the + // consumer so it exits quickly + consumer.wakeup(); + PowerMock.expectLastCall(); + + consumer.close(); + PowerMock.expectLastCall(); + } + + // Note that this can only be called once per test currently + private Capture> expectPolls(final long pollDelayMs) throws Exception { + // Stub out all the consumer stream/iterator responses, which we just want to verify occur, + // but don't care about the exact details here. + EasyMock.expect(consumer.poll(EasyMock.anyLong())).andStubAnswer( + new IAnswer>() { + @Override + public ConsumerRecords answer() throws Throwable { + // "Sleep" so time will progress + time.sleep(pollDelayMs); + ConsumerRecords records = new ConsumerRecords<>( + Collections.singletonMap( + new TopicPartition(TOPIC, PARTITION), + Arrays.asList( + new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, RAW_KEY, RAW_VALUE) + ))); + recordsReturned++; + return records; + } + }); + EasyMock.expect(keyConverter.toCopycatData(TOPIC, RAW_KEY)).andReturn(new SchemaAndValue(KEY_SCHEMA, KEY)).anyTimes(); + EasyMock.expect(valueConverter.toCopycatData(TOPIC, RAW_VALUE)).andReturn(new SchemaAndValue(VALUE_SCHEMA, VALUE)).anyTimes(); + Capture> capturedRecords = EasyMock.newCapture(CaptureType.ALL); + sinkTask.put(EasyMock.capture(capturedRecords)); + EasyMock.expectLastCall().anyTimes(); + return capturedRecords; + } + + private IExpectationSetters expectOnePoll() { + // Currently the SinkTask's put() method will not be invoked unless we provide some data, so instead of + // returning empty data, we return one record. The expectation is that the data will be ignored by the + // response behavior specified using the return value of this method. + EasyMock.expect(consumer.poll(EasyMock.anyLong())).andAnswer( + new IAnswer>() { + @Override + public ConsumerRecords answer() throws Throwable { + // "Sleep" so time will progress + time.sleep(1L); + ConsumerRecords records = new ConsumerRecords<>( + Collections.singletonMap( + new TopicPartition(TOPIC, PARTITION), + Arrays.asList( + new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, RAW_KEY, RAW_VALUE) + ))); + recordsReturned++; + return records; + } + }); + EasyMock.expect(keyConverter.toCopycatData(TOPIC, RAW_KEY)).andReturn(new SchemaAndValue(KEY_SCHEMA, KEY)); + EasyMock.expect(valueConverter.toCopycatData(TOPIC, RAW_VALUE)).andReturn(new SchemaAndValue(VALUE_SCHEMA, VALUE)); + sinkTask.put(EasyMock.anyObject(Collection.class)); + return EasyMock.expectLastCall(); + } + + private Capture expectOffsetFlush(final long expectedMessages, + final RuntimeException flushError, + final Exception consumerCommitError, + final long consumerCommitDelayMs, + final boolean invokeCallback) + throws Exception { + final long finalOffset = FIRST_OFFSET + expectedMessages - 1; + + EasyMock.expect(consumer.assignment()).andReturn(Collections.singleton(TOPIC_PARTITION)); + EasyMock.expect(consumer.position(TOPIC_PARTITION)).andAnswer( + new IAnswer() { + @Override + public Long answer() throws Throwable { + return FIRST_OFFSET + recordsReturned - 1; + } + } + ); + + sinkTask.flush(Collections.singletonMap(TOPIC_PARTITION, new OffsetAndMetadata(finalOffset))); + IExpectationSetters flushExpectation = PowerMock.expectLastCall(); + if (flushError != null) { + flushExpectation.andThrow(flushError).once(); + return null; + } + + final Capture capturedCallback = EasyMock.newCapture(); + final Map offsets = Collections.singletonMap(TOPIC_PARTITION, new OffsetAndMetadata(finalOffset)); + consumer.commitAsync(EasyMock.eq(offsets), + EasyMock.capture(capturedCallback)); + PowerMock.expectLastCall().andAnswer(new IAnswer() { + @Override + public Object answer() throws Throwable { + time.sleep(consumerCommitDelayMs); + if (invokeCallback) + capturedCallback.getValue().onComplete(offsets, consumerCommitError); + return null; + } + }); + return capturedCallback; + } + +} diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/WorkerSourceTaskTest.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/WorkerSourceTaskTest.java new file mode 100644 index 0000000000000..3ff3a6237a26d --- /dev/null +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/WorkerSourceTaskTest.java @@ -0,0 +1,279 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

            + * http://www.apache.org/licenses/LICENSE-2.0 + *

            + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.runtime; + +import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.copycat.cli.WorkerConfig; +import org.apache.kafka.copycat.data.Schema; +import org.apache.kafka.copycat.source.SourceRecord; +import org.apache.kafka.copycat.source.SourceTask; +import org.apache.kafka.copycat.source.SourceTaskContext; +import org.apache.kafka.copycat.storage.Converter; +import org.apache.kafka.copycat.storage.OffsetStorageReader; +import org.apache.kafka.copycat.storage.OffsetStorageWriter; +import org.apache.kafka.copycat.util.Callback; +import org.apache.kafka.copycat.util.ConnectorTaskId; +import org.apache.kafka.copycat.util.ThreadedTest; +import org.easymock.Capture; +import org.easymock.EasyMock; +import org.easymock.IAnswer; +import org.easymock.IExpectationSetters; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.api.easymock.annotation.Mock; +import org.powermock.modules.junit4.PowerMockRunner; +import org.powermock.reflect.Whitebox; + +import java.util.*; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import static org.junit.Assert.*; + +@RunWith(PowerMockRunner.class) +public class WorkerSourceTaskTest extends ThreadedTest { + private static final String TOPIC = "topic"; + private static final Map PARTITION = Collections.singletonMap("key", "partition".getBytes()); + private static final Map OFFSET = Collections.singletonMap("key", 12); + + // Copycat-format data + private static final Schema KEY_SCHEMA = Schema.INT32_SCHEMA; + private static final Integer KEY = -1; + private static final Schema RECORD_SCHEMA = Schema.INT64_SCHEMA; + private static final Long RECORD = 12L; + // Serialized data. The actual format of this data doesn't matter -- we just want to see that the right version + // is used in the right place. + private static final byte[] SERIALIZED_KEY = "converted-key".getBytes(); + private static final byte[] SERIALIZED_RECORD = "converted-record".getBytes(); + + private ConnectorTaskId taskId = new ConnectorTaskId("job", 0); + private WorkerConfig config; + @Mock private SourceTask sourceTask; + @Mock private Converter keyConverter; + @Mock private Converter valueConverter; + @Mock private KafkaProducer producer; + @Mock private OffsetStorageReader offsetReader; + @Mock private OffsetStorageWriter offsetWriter; + private WorkerSourceTask workerTask; + @Mock private Future sendFuture; + + private Capture producerCallbacks; + + private static final Properties EMPTY_TASK_PROPS = new Properties(); + private static final List RECORDS = Arrays.asList( + new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD) + ); + + @Override + public void setup() { + super.setup(); + Properties workerProps = new Properties(); + workerProps.setProperty("key.converter", "org.apache.kafka.copycat.json.JsonConverter"); + workerProps.setProperty("value.converter", "org.apache.kafka.copycat.json.JsonConverter"); + workerProps.setProperty("offset.key.converter", "org.apache.kafka.copycat.json.JsonConverter"); + workerProps.setProperty("offset.value.converter", "org.apache.kafka.copycat.json.JsonConverter"); + workerProps.setProperty("offset.key.converter.schemas.enable", "false"); + workerProps.setProperty("offset.value.converter.schemas.enable", "false"); + config = new WorkerConfig(workerProps); + producerCallbacks = EasyMock.newCapture(); + } + + private void createWorkerTask() { + workerTask = new WorkerSourceTask(taskId, sourceTask, keyConverter, valueConverter, producer, + offsetReader, offsetWriter, config, new SystemTime()); + } + + @Test + public void testPollsInBackground() throws Exception { + createWorkerTask(); + + sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); + EasyMock.expectLastCall(); + sourceTask.start(EMPTY_TASK_PROPS); + EasyMock.expectLastCall(); + + final CountDownLatch pollLatch = expectPolls(10); + // In this test, we don't flush, so nothing goes any further than the offset writer + + sourceTask.stop(); + EasyMock.expectLastCall(); + expectOffsetFlush(true); + + PowerMock.replayAll(); + + workerTask.start(EMPTY_TASK_PROPS); + awaitPolls(pollLatch); + workerTask.stop(); + assertEquals(true, workerTask.awaitStop(1000)); + + PowerMock.verifyAll(); + } + + @Test + public void testCommit() throws Exception { + // Test that the task commits properly when prompted + createWorkerTask(); + + sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); + EasyMock.expectLastCall(); + sourceTask.start(EMPTY_TASK_PROPS); + EasyMock.expectLastCall(); + + // We'll wait for some data, then trigger a flush + final CountDownLatch pollLatch = expectPolls(1); + expectOffsetFlush(true); + + sourceTask.stop(); + EasyMock.expectLastCall(); + expectOffsetFlush(true); + + PowerMock.replayAll(); + + workerTask.start(EMPTY_TASK_PROPS); + awaitPolls(pollLatch); + assertTrue(workerTask.commitOffsets()); + workerTask.stop(); + assertEquals(true, workerTask.awaitStop(1000)); + + PowerMock.verifyAll(); + } + + @Test + public void testCommitFailure() throws Exception { + // Test that the task commits properly when prompted + createWorkerTask(); + + sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); + EasyMock.expectLastCall(); + sourceTask.start(EMPTY_TASK_PROPS); + EasyMock.expectLastCall(); + + // We'll wait for some data, then trigger a flush + final CountDownLatch pollLatch = expectPolls(1); + expectOffsetFlush(false); + + sourceTask.stop(); + EasyMock.expectLastCall(); + expectOffsetFlush(true); + + PowerMock.replayAll(); + + workerTask.start(EMPTY_TASK_PROPS); + awaitPolls(pollLatch); + assertFalse(workerTask.commitOffsets()); + workerTask.stop(); + assertEquals(true, workerTask.awaitStop(1000)); + + PowerMock.verifyAll(); + } + + @Test + public void testSendRecordsConvertsData() throws Exception { + createWorkerTask(); + + List records = new ArrayList<>(); + // Can just use the same record for key and value + records.add(new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD)); + + Capture> sent = expectSendRecord(); + + PowerMock.replayAll(); + + Whitebox.invokeMethod(workerTask, "sendRecords", records); + assertEquals(SERIALIZED_KEY, sent.getValue().key()); + assertEquals(SERIALIZED_RECORD, sent.getValue().value()); + + PowerMock.verifyAll(); + } + + + private CountDownLatch expectPolls(int count) throws InterruptedException { + final CountDownLatch latch = new CountDownLatch(count); + // Note that we stub these to allow any number of calls because the thread will continue to + // run. The count passed in + latch returned just makes sure we get *at least* that number of + // calls + EasyMock.expect(sourceTask.poll()) + .andStubAnswer(new IAnswer>() { + @Override + public List answer() throws Throwable { + latch.countDown(); + return RECORDS; + } + }); + // Fallout of the poll() call + expectSendRecord(); + return latch; + } + + private Capture> expectSendRecord() throws InterruptedException { + EasyMock.expect(keyConverter.fromCopycatData(TOPIC, KEY_SCHEMA, KEY)).andStubReturn(SERIALIZED_KEY); + EasyMock.expect(valueConverter.fromCopycatData(TOPIC, RECORD_SCHEMA, RECORD)).andStubReturn(SERIALIZED_RECORD); + + Capture> sent = EasyMock.newCapture(); + // 1. Converted data passed to the producer, which will need callbacks invoked for flush to work + EasyMock.expect( + producer.send(EasyMock.capture(sent), + EasyMock.capture(producerCallbacks))) + .andStubAnswer(new IAnswer>() { + @Override + public Future answer() throws Throwable { + synchronized (producerCallbacks) { + for (org.apache.kafka.clients.producer.Callback cb : producerCallbacks.getValues()) { + cb.onCompletion(new RecordMetadata(new TopicPartition("foo", 0), 0, 0), null); + } + producerCallbacks.reset(); + } + return sendFuture; + } + }); + // 2. Offset data is passed to the offset storage. + offsetWriter.offset(PARTITION, OFFSET); + PowerMock.expectLastCall().anyTimes(); + + return sent; + } + + private void awaitPolls(CountDownLatch latch) throws InterruptedException { + latch.await(1000, TimeUnit.MILLISECONDS); + } + + @SuppressWarnings("unchecked") + private void expectOffsetFlush(boolean succeed) throws Exception { + EasyMock.expect(offsetWriter.beginFlush()).andReturn(true); + Future flushFuture = PowerMock.createMock(Future.class); + EasyMock.expect(offsetWriter.doFlush(EasyMock.anyObject(Callback.class))).andReturn(flushFuture); + // Should throw for failure + IExpectationSetters futureGetExpect = EasyMock.expect( + flushFuture.get(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class))); + if (succeed) { + futureGetExpect.andReturn(null); + } else { + futureGetExpect.andThrow(new TimeoutException()); + offsetWriter.cancelFlush(); + PowerMock.expectLastCall(); + } + } + +} diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/WorkerTest.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/WorkerTest.java new file mode 100644 index 0000000000000..e75d2f90d1ad4 --- /dev/null +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/WorkerTest.java @@ -0,0 +1,201 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

            + * http://www.apache.org/licenses/LICENSE-2.0 + *

            + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.runtime; + +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.copycat.cli.WorkerConfig; +import org.apache.kafka.copycat.errors.CopycatException; +import org.apache.kafka.copycat.source.SourceRecord; +import org.apache.kafka.copycat.source.SourceTask; +import org.apache.kafka.copycat.storage.*; +import org.apache.kafka.copycat.util.ConnectorTaskId; +import org.apache.kafka.copycat.util.MockTime; +import org.apache.kafka.copycat.util.ThreadedTest; +import org.easymock.EasyMock; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.List; +import java.util.Map; +import java.util.Properties; + +@RunWith(PowerMockRunner.class) +@PrepareForTest(Worker.class) +@PowerMockIgnore("javax.management.*") +public class WorkerTest extends ThreadedTest { + + private ConnectorTaskId taskId = new ConnectorTaskId("job", 0); + private WorkerConfig config; + private Worker worker; + private OffsetBackingStore offsetBackingStore = PowerMock.createMock(OffsetBackingStore.class); + + @Before + public void setup() { + super.setup(); + + Properties workerProps = new Properties(); + workerProps.setProperty("key.converter", "org.apache.kafka.copycat.json.JsonConverter"); + workerProps.setProperty("value.converter", "org.apache.kafka.copycat.json.JsonConverter"); + workerProps.setProperty("offset.key.converter", "org.apache.kafka.copycat.json.JsonConverter"); + workerProps.setProperty("offset.value.converter", "org.apache.kafka.copycat.json.JsonConverter"); + workerProps.setProperty("offset.key.converter.schemas.enable", "false"); + workerProps.setProperty("offset.value.converter.schemas.enable", "false"); + config = new WorkerConfig(workerProps); + } + + @Test + public void testAddRemoveTask() throws Exception { + offsetBackingStore.configure(EasyMock.anyObject(Map.class)); + EasyMock.expectLastCall(); + offsetBackingStore.start(); + EasyMock.expectLastCall(); + + ConnectorTaskId taskId = new ConnectorTaskId("job", 0); + + // Create + TestSourceTask task = PowerMock.createMock(TestSourceTask.class); + WorkerSourceTask workerTask = PowerMock.createMock(WorkerSourceTask.class); + + PowerMock.mockStatic(Worker.class); + PowerMock.expectPrivate(Worker.class, "instantiateTask", TestSourceTask.class.getName()).andReturn(task); + + PowerMock.expectNew( + WorkerSourceTask.class, EasyMock.eq(taskId), EasyMock.eq(task), + EasyMock.anyObject(Converter.class), + EasyMock.anyObject(Converter.class), + EasyMock.anyObject(KafkaProducer.class), + EasyMock.anyObject(OffsetStorageReader.class), + EasyMock.anyObject(OffsetStorageWriter.class), + EasyMock.anyObject(WorkerConfig.class), + EasyMock.anyObject(Time.class)) + .andReturn(workerTask); + Properties origProps = new Properties(); + workerTask.start(origProps); + EasyMock.expectLastCall(); + + // Remove + workerTask.stop(); + EasyMock.expectLastCall(); + EasyMock.expect(workerTask.awaitStop(EasyMock.anyLong())).andStubReturn(true); + workerTask.close(); + EasyMock.expectLastCall(); + + offsetBackingStore.stop(); + EasyMock.expectLastCall(); + + PowerMock.replayAll(); + + worker = new Worker(new MockTime(), config, offsetBackingStore); + worker.start(); + worker.addTask(taskId, TestSourceTask.class.getName(), origProps); + worker.stopTask(taskId); + // Nothing should be left, so this should effectively be a nop + worker.stop(); + + PowerMock.verifyAll(); + } + + @Test(expected = CopycatException.class) + public void testStopInvalidTask() { + offsetBackingStore.configure(EasyMock.anyObject(Map.class)); + EasyMock.expectLastCall(); + offsetBackingStore.start(); + EasyMock.expectLastCall(); + + PowerMock.replayAll(); + + worker = new Worker(new MockTime(), config, offsetBackingStore); + worker.start(); + + worker.stopTask(taskId); + } + + @Test + public void testCleanupTasksOnStop() throws Exception { + offsetBackingStore.configure(EasyMock.anyObject(Map.class)); + EasyMock.expectLastCall(); + offsetBackingStore.start(); + EasyMock.expectLastCall(); + + // Create + TestSourceTask task = PowerMock.createMock(TestSourceTask.class); + WorkerSourceTask workerTask = PowerMock.createMock(WorkerSourceTask.class); + + PowerMock.mockStatic(Worker.class); + PowerMock.expectPrivate(Worker.class, "instantiateTask", TestSourceTask.class.getName()).andReturn(task); + + PowerMock.expectNew( + WorkerSourceTask.class, EasyMock.eq(taskId), EasyMock.eq(task), + EasyMock.anyObject(Converter.class), + EasyMock.anyObject(Converter.class), + EasyMock.anyObject(KafkaProducer.class), + EasyMock.anyObject(OffsetStorageReader.class), + EasyMock.anyObject(OffsetStorageWriter.class), + EasyMock.anyObject(WorkerConfig.class), + EasyMock.anyObject(Time.class)) + .andReturn(workerTask); + Properties origProps = new Properties(); + workerTask.start(origProps); + EasyMock.expectLastCall(); + + // Remove on Worker.stop() + workerTask.stop(); + EasyMock.expectLastCall(); + EasyMock.expect(workerTask.awaitStop(EasyMock.anyLong())).andReturn(true); + // Note that in this case we *do not* commit offsets since it's an unclean shutdown + workerTask.close(); + EasyMock.expectLastCall(); + + offsetBackingStore.stop(); + EasyMock.expectLastCall(); + + PowerMock.replayAll(); + + worker = new Worker(new MockTime(), config, offsetBackingStore); + worker.start(); + worker.addTask(taskId, TestSourceTask.class.getName(), origProps); + worker.stop(); + + PowerMock.verifyAll(); + } + + + private static class TestSourceTask extends SourceTask { + public TestSourceTask() { + } + + @Override + public void start(Properties props) { + } + + @Override + public List poll() throws InterruptedException { + return null; + } + + @Override + public void stop() { + } + } +} diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/standalone/StandaloneHerderTest.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/standalone/StandaloneHerderTest.java new file mode 100644 index 0000000000000..477893bb1fb3f --- /dev/null +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/standalone/StandaloneHerderTest.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.copycat.runtime.standalone; + +import org.apache.kafka.copycat.connector.Connector; +import org.apache.kafka.copycat.connector.Task; +import org.apache.kafka.copycat.runtime.ConnectorConfig; +import org.apache.kafka.copycat.runtime.Worker; +import org.apache.kafka.copycat.sink.SinkConnector; +import org.apache.kafka.copycat.sink.SinkTask; +import org.apache.kafka.copycat.source.SourceConnector; +import org.apache.kafka.copycat.source.SourceTask; +import org.apache.kafka.copycat.util.Callback; +import org.apache.kafka.copycat.util.ConnectorTaskId; +import org.apache.kafka.copycat.util.FutureCallback; +import org.easymock.EasyMock; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.api.easymock.annotation.Mock; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.Arrays; +import java.util.Properties; +import java.util.concurrent.TimeUnit; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({StandaloneHerder.class}) +@PowerMockIgnore("javax.management.*") +public class StandaloneHerderTest { + private static final String CONNECTOR_NAME = "test"; + private static final String TOPICS_LIST_STR = "topic1,topic2"; + + private StandaloneHerder herder; + @Mock protected Worker worker; + private Connector connector; + @Mock protected Callback createCallback; + + private Properties connectorProps; + private Properties taskProps; + + @Before + public void setup() { + worker = PowerMock.createMock(Worker.class); + herder = new StandaloneHerder(worker); + + connectorProps = new Properties(); + connectorProps.setProperty(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME); + connectorProps.setProperty(SinkConnector.TOPICS_CONFIG, TOPICS_LIST_STR); + PowerMock.mockStatic(StandaloneHerder.class); + + // These can be anything since connectors can pass along whatever they want. + taskProps = new Properties(); + taskProps.setProperty("foo", "bar"); + } + + @Test + public void testCreateSourceConnector() throws Exception { + connector = PowerMock.createMock(BogusSourceClass.class); + expectAdd(BogusSourceClass.class, BogusSourceTask.class, false); + PowerMock.replayAll(); + + herder.addConnector(connectorProps, createCallback); + + PowerMock.verifyAll(); + } + + @Test + public void testCreateSinkConnector() throws Exception { + connector = PowerMock.createMock(BogusSinkClass.class); + expectAdd(BogusSinkClass.class, BogusSinkTask.class, true); + + PowerMock.replayAll(); + + herder.addConnector(connectorProps, createCallback); + + PowerMock.verifyAll(); + } + + @Test + public void testDestroyConnector() throws Exception { + connector = PowerMock.createMock(BogusSourceClass.class); + expectAdd(BogusSourceClass.class, BogusSourceTask.class, false); + expectDestroy(); + PowerMock.replayAll(); + + herder.addConnector(connectorProps, createCallback); + FutureCallback futureCb = new FutureCallback<>(new Callback() { + @Override + public void onCompletion(Throwable error, Void result) { + + } + }); + herder.deleteConnector(CONNECTOR_NAME, futureCb); + futureCb.get(1000L, TimeUnit.MILLISECONDS); + PowerMock.verifyAll(); + } + + + private void expectAdd(Class connClass, + Class taskClass, + boolean sink) throws Exception { + expectCreate(connClass, taskClass, sink, true); + } + + private void expectRestore(Class connClass, + Class taskClass) throws Exception { + // Restore never uses a callback. These tests always use sources + expectCreate(connClass, taskClass, false, false); + } + + private void expectCreate(Class connClass, + Class taskClass, + boolean sink, boolean expectCallback) throws Exception { + connectorProps.setProperty(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connClass.getName()); + + PowerMock.expectPrivate(StandaloneHerder.class, "instantiateConnector", connClass.getName()) + .andReturn(connector); + if (expectCallback) { + createCallback.onCompletion(null, CONNECTOR_NAME); + PowerMock.expectLastCall(); + } + + connector.initialize(EasyMock.anyObject(StandaloneConnectorContext.class)); + PowerMock.expectLastCall(); + connector.start(new Properties()); + PowerMock.expectLastCall(); + + // Just return the connector properties for the individual task we generate by default + EasyMock.>expect(connector.taskClass()).andReturn(taskClass); + + EasyMock.expect(connector.taskConfigs(ConnectorConfig.TASKS_MAX_DEFAULT)) + .andReturn(Arrays.asList(taskProps)); + // And we should instantiate the tasks. For a sink task, we should see added properties for + // the input topic partitions + Properties generatedTaskProps = new Properties(); + generatedTaskProps.putAll(taskProps); + if (sink) + generatedTaskProps.setProperty(SinkTask.TOPICS_CONFIG, TOPICS_LIST_STR); + worker.addTask(new ConnectorTaskId(CONNECTOR_NAME, 0), taskClass.getName(), generatedTaskProps); + PowerMock.expectLastCall(); + } + + private void expectStop() { + worker.stopTask(new ConnectorTaskId(CONNECTOR_NAME, 0)); + EasyMock.expectLastCall(); + connector.stop(); + EasyMock.expectLastCall(); + } + + private void expectDestroy() { + expectStop(); + } + + // We need to use a real class here due to some issue with mocking java.lang.Class + private abstract class BogusSourceClass extends SourceConnector { + } + + private abstract class BogusSourceTask extends SourceTask { + } + + private abstract class BogusSinkClass extends SinkConnector { + } + + private abstract class BogusSinkTask extends SourceTask { + } + +} diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/storage/FileOffsetBackingStoreTest.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/storage/FileOffsetBackingStoreTest.java new file mode 100644 index 0000000000000..2976c0a7398df --- /dev/null +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/storage/FileOffsetBackingStoreTest.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.copycat.storage; + +import org.apache.kafka.copycat.util.Callback; +import org.easymock.EasyMock; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.powermock.api.easymock.PowerMock; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +public class FileOffsetBackingStoreTest { + + FileOffsetBackingStore store; + Map props; + File tempFile; + + private static Map firstSet = new HashMap<>(); + + static { + firstSet.put(buffer("key"), buffer("value")); + firstSet.put(null, null); + } + + @Before + public void setup() throws IOException { + store = new FileOffsetBackingStore(); + tempFile = File.createTempFile("fileoffsetbackingstore", null); + props = new HashMap<>(); + props.put(FileOffsetBackingStore.OFFSET_STORAGE_FILE_FILENAME_CONFIG, tempFile.getAbsolutePath()); + store.configure(props); + store.start(); + } + + @After + public void teardown() { + tempFile.delete(); + } + + @Test + public void testGetSet() throws Exception { + Callback setCallback = expectSuccessfulSetCallback(); + Callback> getCallback = expectSuccessfulGetCallback(); + PowerMock.replayAll(); + + store.set(firstSet, setCallback).get(); + + Map values = store.get(Arrays.asList(buffer("key"), buffer("bad")), getCallback).get(); + assertEquals(buffer("value"), values.get(buffer("key"))); + assertEquals(null, values.get(buffer("bad"))); + + PowerMock.verifyAll(); + } + + @Test + public void testSaveRestore() throws Exception { + Callback setCallback = expectSuccessfulSetCallback(); + Callback> getCallback = expectSuccessfulGetCallback(); + PowerMock.replayAll(); + + store.set(firstSet, setCallback).get(); + store.stop(); + + // Restore into a new store to ensure correct reload from scratch + FileOffsetBackingStore restore = new FileOffsetBackingStore(); + restore.configure(props); + restore.start(); + Map values = restore.get(Arrays.asList(buffer("key")), getCallback).get(); + assertEquals(buffer("value"), values.get(buffer("key"))); + + PowerMock.verifyAll(); + } + + private static ByteBuffer buffer(String v) { + return ByteBuffer.wrap(v.getBytes()); + } + + private Callback expectSuccessfulSetCallback() { + @SuppressWarnings("unchecked") + Callback setCallback = PowerMock.createMock(Callback.class); + setCallback.onCompletion(EasyMock.isNull(Throwable.class), EasyMock.isNull(Void.class)); + PowerMock.expectLastCall(); + return setCallback; + } + + @SuppressWarnings("unchecked") + private Callback> expectSuccessfulGetCallback() { + Callback> getCallback = PowerMock.createMock(Callback.class); + getCallback.onCompletion(EasyMock.isNull(Throwable.class), EasyMock.anyObject(Map.class)); + PowerMock.expectLastCall(); + return getCallback; + } +} diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/storage/KafkaOffsetBackingStoreTest.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/storage/KafkaOffsetBackingStoreTest.java new file mode 100644 index 0000000000000..6a3eec3e48469 --- /dev/null +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/storage/KafkaOffsetBackingStoreTest.java @@ -0,0 +1,458 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.storage; + +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.MockConsumer; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.LeaderNotAvailableException; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.copycat.errors.CopycatException; +import org.apache.kafka.copycat.util.Callback; +import org.apache.kafka.copycat.util.TestFuture; +import org.easymock.Capture; +import org.easymock.EasyMock; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.api.easymock.annotation.Mock; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; +import org.powermock.reflect.Whitebox; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.apache.kafka.copycat.util.ByteArrayProducerRecordEquals.eqProducerRecord; + +@RunWith(PowerMockRunner.class) +@PrepareForTest(KafkaOffsetBackingStore.class) +@PowerMockIgnore("javax.management.*") +public class KafkaOffsetBackingStoreTest { + private static final String TOPIC = "copycat-offsets"; + private static final TopicPartition TP0 = new TopicPartition(TOPIC, 0); + private static final TopicPartition TP1 = new TopicPartition(TOPIC, 1); + private static final Map DEFAULT_PROPS = new HashMap<>(); + static { + DEFAULT_PROPS.put(KafkaOffsetBackingStore.OFFSET_STORAGE_TOPIC_CONFIG, TOPIC); + DEFAULT_PROPS.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "broker1:9092,broker2:9093"); + } + private static final Set CONSUMER_ASSIGNMENT = new HashSet<>(Arrays.asList(TP0, TP1)); + private static final Map FIRST_SET = new HashMap<>(); + static { + FIRST_SET.put(buffer("key"), buffer("value")); + FIRST_SET.put(null, null); + } + + + private static final Node LEADER = new Node(1, "broker1", 9092); + private static final Node REPLICA = new Node(1, "broker2", 9093); + + private static final PartitionInfo TPINFO0 = new PartitionInfo(TOPIC, 0, LEADER, new Node[]{REPLICA}, new Node[]{REPLICA}); + private static final PartitionInfo TPINFO1 = new PartitionInfo(TOPIC, 1, LEADER, new Node[]{REPLICA}, new Node[]{REPLICA}); + + private static final ByteBuffer TP0_KEY = buffer("TP0KEY"); + private static final ByteBuffer TP1_KEY = buffer("TP1KEY"); + private static final ByteBuffer TP0_VALUE = buffer("VAL0"); + private static final ByteBuffer TP1_VALUE = buffer("VAL1"); + private static final ByteBuffer TP0_VALUE_NEW = buffer("VAL0_NEW"); + private static final ByteBuffer TP1_VALUE_NEW = buffer("VAL1_NEW"); + + private KafkaOffsetBackingStore store; + + @Mock private KafkaProducer producer; + private MockConsumer consumer; + + @Before + public void setUp() throws Exception { + store = PowerMock.createPartialMockAndInvokeDefaultConstructor(KafkaOffsetBackingStore.class, new String[]{"createConsumer", "createProducer"}); + consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + consumer.updatePartitions(TOPIC, Arrays.asList(TPINFO0, TPINFO1)); + Map beginningOffsets = new HashMap<>(); + beginningOffsets.put(TP0, 0L); + beginningOffsets.put(TP1, 0L); + consumer.updateBeginningOffsets(beginningOffsets); + } + + @Test(expected = CopycatException.class) + public void testMissingTopic() { + store = new KafkaOffsetBackingStore(); + store.configure(Collections.emptyMap()); + } + + @Test + public void testStartStop() throws Exception { + expectStart(); + expectStop(); + + PowerMock.replayAll(); + + store.configure(DEFAULT_PROPS); + + Map endOffsets = new HashMap<>(); + endOffsets.put(TP0, 0L); + endOffsets.put(TP1, 0L); + consumer.updateEndOffsets(endOffsets); + store.start(); + assertEquals(CONSUMER_ASSIGNMENT, consumer.assignment()); + + store.stop(); + + assertFalse(Whitebox.getInternalState(store, "thread").isAlive()); + assertTrue(consumer.closed()); + PowerMock.verifyAll(); + } + + @Test + public void testReloadOnStart() throws Exception { + expectStart(); + expectStop(); + + PowerMock.replayAll(); + + store.configure(DEFAULT_PROPS); + + Map endOffsets = new HashMap<>(); + endOffsets.put(TP0, 1L); + endOffsets.put(TP1, 1L); + consumer.updateEndOffsets(endOffsets); + Thread startConsumerOpsThread = new Thread("start-consumer-ops-thread") { + @Override + public void run() { + // Needs to seek to end to find end offsets + consumer.waitForPoll(10000); + + // Should keep polling until it reaches current log end offset for all partitions + consumer.waitForPollThen(new Runnable() { + @Override + public void run() { + consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, TP0_KEY.array(), TP0_VALUE.array())); + } + }, 10000); + + consumer.waitForPollThen(new Runnable() { + @Override + public void run() { + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, TP1_KEY.array(), TP1_VALUE.array())); + } + }, 10000); + } + }; + startConsumerOpsThread.start(); + store.start(); + startConsumerOpsThread.join(10000); + assertFalse(startConsumerOpsThread.isAlive()); + assertEquals(CONSUMER_ASSIGNMENT, consumer.assignment()); + HashMap data = Whitebox.getInternalState(store, "data"); + assertEquals(TP0_VALUE, data.get(TP0_KEY)); + assertEquals(TP1_VALUE, data.get(TP1_KEY)); + + store.stop(); + + assertFalse(Whitebox.getInternalState(store, "thread").isAlive()); + assertTrue(consumer.closed()); + PowerMock.verifyAll(); + } + + @Test + public void testGetSet() throws Exception { + expectStart(); + TestFuture tp0Future = new TestFuture<>(); + ProducerRecord tp0Record = new ProducerRecord<>(TOPIC, TP0_KEY.array(), TP0_VALUE.array()); + Capture callback1 = EasyMock.newCapture(); + EasyMock.expect(producer.send(eqProducerRecord(tp0Record), EasyMock.capture(callback1))).andReturn(tp0Future); + TestFuture tp1Future = new TestFuture<>(); + ProducerRecord tp1Record = new ProducerRecord<>(TOPIC, TP1_KEY.array(), TP1_VALUE.array()); + Capture callback2 = EasyMock.newCapture(); + EasyMock.expect(producer.send(eqProducerRecord(tp1Record), EasyMock.capture(callback2))).andReturn(tp1Future); + + expectStop(); + + PowerMock.replayAll(); + + store.configure(DEFAULT_PROPS); + + Map endOffsets = new HashMap<>(); + endOffsets.put(TP0, 0L); + endOffsets.put(TP1, 0L); + consumer.updateEndOffsets(endOffsets); + Thread startConsumerOpsThread = new Thread("start-consumer-ops-thread") { + @Override + public void run() { + // Should keep polling until it has partition info + consumer.waitForPollThen(new Runnable() { + @Override + public void run() { + consumer.seek(TP0, 0); + consumer.seek(TP1, 0); + } + }, 10000); + } + }; + startConsumerOpsThread.start(); + store.start(); + startConsumerOpsThread.join(10000); + assertFalse(startConsumerOpsThread.isAlive()); + assertEquals(CONSUMER_ASSIGNMENT, consumer.assignment()); + + Map toSet = new HashMap<>(); + toSet.put(TP0_KEY, TP0_VALUE); + toSet.put(TP1_KEY, TP1_VALUE); + final AtomicBoolean invoked = new AtomicBoolean(false); + Future setFuture = store.set(toSet, new Callback() { + @Override + public void onCompletion(Throwable error, Void result) { + invoked.set(true); + } + }); + assertFalse(setFuture.isDone()); + tp1Future.resolve((RecordMetadata) null); // Output not used, so safe to not return a real value for testing + assertFalse(setFuture.isDone()); + tp0Future.resolve((RecordMetadata) null); + // Out of order callbacks shouldn't matter, should still require all to be invoked before invoking the callback + // for the store's set callback + callback2.getValue().onCompletion(null, null); + assertFalse(invoked.get()); + callback1.getValue().onCompletion(null, null); + setFuture.get(10000, TimeUnit.MILLISECONDS); + assertTrue(invoked.get()); + + // Getting data should continue to return old data... + final AtomicBoolean getInvokedAndPassed = new AtomicBoolean(false); + store.get(Arrays.asList(TP0_KEY, TP1_KEY), new Callback>() { + @Override + public void onCompletion(Throwable error, Map result) { + // Since we didn't read them yet, these will be null + assertEquals(null, result.get(TP0_KEY)); + assertEquals(null, result.get(TP1_KEY)); + getInvokedAndPassed.set(true); + } + }).get(10000, TimeUnit.MILLISECONDS); + assertTrue(getInvokedAndPassed.get()); + + // Until the consumer gets the new data + Thread readNewDataThread = new Thread("read-new-data-thread") { + @Override + public void run() { + // Should keep polling until it reaches current log end offset for all partitions + consumer.waitForPollThen(new Runnable() { + @Override + public void run() { + consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, TP0_KEY.array(), TP0_VALUE_NEW.array())); + } + }, 10000); + + consumer.waitForPollThen(new Runnable() { + @Override + public void run() { + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, TP1_KEY.array(), TP1_VALUE_NEW.array())); + } + }, 10000); + } + }; + readNewDataThread.start(); + readNewDataThread.join(10000); + assertFalse(readNewDataThread.isAlive()); + + // And now the new data should be returned + final AtomicBoolean finalGetInvokedAndPassed = new AtomicBoolean(false); + store.get(Arrays.asList(TP0_KEY, TP1_KEY), new Callback>() { + @Override + public void onCompletion(Throwable error, Map result) { + assertEquals(TP0_VALUE_NEW, result.get(TP0_KEY)); + assertEquals(TP1_VALUE_NEW, result.get(TP1_KEY)); + finalGetInvokedAndPassed.set(true); + } + }).get(10000, TimeUnit.MILLISECONDS); + assertTrue(finalGetInvokedAndPassed.get()); + + store.stop(); + + assertFalse(Whitebox.getInternalState(store, "thread").isAlive()); + assertTrue(consumer.closed()); + PowerMock.verifyAll(); + } + + @Test + public void testConsumerError() throws Exception { + expectStart(); + expectStop(); + + PowerMock.replayAll(); + + store.configure(DEFAULT_PROPS); + + Map endOffsets = new HashMap<>(); + endOffsets.put(TP0, 1L); + endOffsets.put(TP1, 1L); + consumer.updateEndOffsets(endOffsets); + Thread startConsumerOpsThread = new Thread("start-consumer-ops-thread") { + @Override + public void run() { + // Trigger exception + consumer.waitForPollThen(new Runnable() { + @Override + public void run() { + consumer.setException(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.exception()); + } + }, 10000); + + // Should keep polling until it reaches current log end offset for all partitions + consumer.waitForPollThen(new Runnable() { + @Override + public void run() { + consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, TP0_KEY.array(), TP0_VALUE_NEW.array())); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, TP0_KEY.array(), TP0_VALUE_NEW.array())); + } + }, 10000); + } + }; + startConsumerOpsThread.start(); + store.start(); + startConsumerOpsThread.join(10000); + assertFalse(startConsumerOpsThread.isAlive()); + assertEquals(CONSUMER_ASSIGNMENT, consumer.assignment()); + + store.stop(); + + assertFalse(Whitebox.getInternalState(store, "thread").isAlive()); + assertTrue(consumer.closed()); + PowerMock.verifyAll(); + } + + @Test + public void testProducerError() throws Exception { + expectStart(); + TestFuture tp0Future = new TestFuture<>(); + ProducerRecord tp0Record = new ProducerRecord<>(TOPIC, TP0_KEY.array(), TP0_VALUE.array()); + Capture callback1 = EasyMock.newCapture(); + EasyMock.expect(producer.send(eqProducerRecord(tp0Record), EasyMock.capture(callback1))).andReturn(tp0Future); + TestFuture tp1Future = new TestFuture<>(); + ProducerRecord tp1Record = new ProducerRecord<>(TOPIC, TP1_KEY.array(), TP1_VALUE.array()); + Capture callback2 = EasyMock.newCapture(); + EasyMock.expect(producer.send(eqProducerRecord(tp1Record), EasyMock.capture(callback2))).andReturn(tp1Future); + + expectStop(); + + PowerMock.replayAll(); + + store.configure(DEFAULT_PROPS); + + Map endOffsets = new HashMap<>(); + endOffsets.put(TP0, 0L); + endOffsets.put(TP1, 0L); + consumer.updateEndOffsets(endOffsets); + Thread startConsumerOpsThread = new Thread("start-consumer-ops-thread") { + @Override + public void run() { + // Should keep polling until it has partition info + consumer.waitForPollThen(new Runnable() { + @Override + public void run() { + consumer.seek(TP0, 0); + consumer.seek(TP1, 0); + } + }, 10000); + } + }; + startConsumerOpsThread.start(); + store.start(); + startConsumerOpsThread.join(10000); + assertFalse(startConsumerOpsThread.isAlive()); + assertEquals(CONSUMER_ASSIGNMENT, consumer.assignment()); + + Map toSet = new HashMap<>(); + toSet.put(TP0_KEY, TP0_VALUE); + toSet.put(TP1_KEY, TP1_VALUE); + final AtomicReference setException = new AtomicReference<>(); + Future setFuture = store.set(toSet, new Callback() { + @Override + public void onCompletion(Throwable error, Void result) { + assertNull(setException.get()); // Should only be invoked once + setException.set(error); + } + }); + assertFalse(setFuture.isDone()); + KafkaException exc = new LeaderNotAvailableException("Error"); + tp1Future.resolve(exc); + callback2.getValue().onCompletion(null, exc); + // One failure should resolve the future immediately + try { + setFuture.get(10000, TimeUnit.MILLISECONDS); + fail("Should have see ExecutionException"); + } catch (ExecutionException e) { + // expected + } + assertNotNull(setException.get()); + + // Callbacks can continue to arrive + tp0Future.resolve((RecordMetadata) null); + callback1.getValue().onCompletion(null, null); + + store.stop(); + + assertFalse(Whitebox.getInternalState(store, "thread").isAlive()); + assertTrue(consumer.closed()); + PowerMock.verifyAll(); + } + + + private void expectStart() throws Exception { + PowerMock.expectPrivate(store, "createProducer") + .andReturn(producer); + PowerMock.expectPrivate(store, "createConsumer") + .andReturn(consumer); + } + + private void expectStop() { + producer.close(); + PowerMock.expectLastCall(); + // MockConsumer close is checked after test. + } + + private static ByteBuffer buffer(String v) { + return ByteBuffer.wrap(v.getBytes()); + } + +} diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/storage/OffsetStorageWriterTest.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/storage/OffsetStorageWriterTest.java new file mode 100644 index 0000000000000..e33ecd0d99476 --- /dev/null +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/storage/OffsetStorageWriterTest.java @@ -0,0 +1,235 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

            + * http://www.apache.org/licenses/LICENSE-2.0 + *

            + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.storage; + +import org.apache.kafka.copycat.errors.CopycatException; +import org.apache.kafka.copycat.util.Callback; +import org.easymock.Capture; +import org.easymock.EasyMock; +import org.easymock.IAnswer; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.api.easymock.annotation.Mock; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.*; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +@RunWith(PowerMockRunner.class) +public class OffsetStorageWriterTest { + private static final String NAMESPACE = "namespace"; + // Copycat format - any types should be accepted here + private static final Map OFFSET_KEY = Collections.singletonMap("key", "key"); + private static final List OFFSET_KEY_WRAPPED = Arrays.asList(NAMESPACE, OFFSET_KEY); + private static final Map OFFSET_VALUE = Collections.singletonMap("key", 12); + + // Serialized + private static final byte[] OFFSET_KEY_SERIALIZED = "key-serialized".getBytes(); + private static final byte[] OFFSET_VALUE_SERIALIZED = "value-serialized".getBytes(); + private static final Map OFFSETS_SERIALIZED + = Collections.singletonMap(ByteBuffer.wrap(OFFSET_KEY_SERIALIZED), + ByteBuffer.wrap(OFFSET_VALUE_SERIALIZED)); + + @Mock private OffsetBackingStore store; + @Mock private Converter keyConverter; + @Mock private Converter valueConverter; + private OffsetStorageWriter writer; + + private static Exception exception = new RuntimeException("error"); + + private ExecutorService service; + + @Before + public void setup() { + writer = new OffsetStorageWriter(store, NAMESPACE, keyConverter, valueConverter); + service = Executors.newFixedThreadPool(1); + } + + @After + public void teardown() { + service.shutdownNow(); + } + + @Test + public void testWriteFlush() throws Exception { + @SuppressWarnings("unchecked") + Callback callback = PowerMock.createMock(Callback.class); + expectStore(callback, false); + + PowerMock.replayAll(); + + writer.offset(OFFSET_KEY, OFFSET_VALUE); + + assertTrue(writer.beginFlush()); + writer.doFlush(callback).get(1000, TimeUnit.MILLISECONDS); + + PowerMock.verifyAll(); + } + + @Test + public void testNoOffsetsToFlush() { + // If no offsets are flushed, we should finish immediately and not have made any calls to the + // underlying storage layer + + PowerMock.replayAll(); + + // Should not return a future + assertFalse(writer.beginFlush()); + + PowerMock.verifyAll(); + } + + @Test + public void testFlushFailureReplacesOffsets() throws Exception { + // When a flush fails, we shouldn't just lose the offsets. Instead, they should be restored + // such that a subsequent flush will write them. + + @SuppressWarnings("unchecked") + final Callback callback = PowerMock.createMock(Callback.class); + // First time the write fails + expectStore(callback, true); + // Second time it succeeds + expectStore(callback, false); + // Third time it has no data to flush so we won't get past beginFlush() + + PowerMock.replayAll(); + + writer.offset(OFFSET_KEY, OFFSET_VALUE); + assertTrue(writer.beginFlush()); + writer.doFlush(callback).get(1000, TimeUnit.MILLISECONDS); + assertTrue(writer.beginFlush()); + writer.doFlush(callback).get(1000, TimeUnit.MILLISECONDS); + assertFalse(writer.beginFlush()); + + PowerMock.verifyAll(); + } + + @Test(expected = CopycatException.class) + public void testAlreadyFlushing() throws Exception { + @SuppressWarnings("unchecked") + final Callback callback = PowerMock.createMock(Callback.class); + // Trigger the send, but don't invoke the callback so we'll still be mid-flush + CountDownLatch allowStoreCompleteCountdown = new CountDownLatch(1); + expectStore(null, false, allowStoreCompleteCountdown); + + PowerMock.replayAll(); + + writer.offset(OFFSET_KEY, OFFSET_VALUE); + assertTrue(writer.beginFlush()); + writer.doFlush(callback); + assertTrue(writer.beginFlush()); // should throw + + PowerMock.verifyAll(); + } + + @Test + public void testCancelBeforeAwaitFlush() { + PowerMock.replayAll(); + + writer.offset(OFFSET_KEY, OFFSET_VALUE); + assertTrue(writer.beginFlush()); + writer.cancelFlush(); + + PowerMock.verifyAll(); + } + + @Test + public void testCancelAfterAwaitFlush() throws Exception { + @SuppressWarnings("unchecked") + Callback callback = PowerMock.createMock(Callback.class); + CountDownLatch allowStoreCompleteCountdown = new CountDownLatch(1); + // In this test, the write should be cancelled so the callback will not be invoked and is not + // passed to the expectStore call + expectStore(null, false, allowStoreCompleteCountdown); + + PowerMock.replayAll(); + + writer.offset(OFFSET_KEY, OFFSET_VALUE); + assertTrue(writer.beginFlush()); + // Start the flush, then immediately cancel before allowing the mocked store request to finish + Future flushFuture = writer.doFlush(callback); + writer.cancelFlush(); + allowStoreCompleteCountdown.countDown(); + flushFuture.get(1000, TimeUnit.MILLISECONDS); + + PowerMock.verifyAll(); + } + + private void expectStore(final Callback callback, final boolean fail) { + expectStore(callback, fail, null); + } + + /** + * Expect a request to store data to the underlying OffsetBackingStore. + * + * @param callback the callback to invoke when completed, or null if the callback isn't + * expected to be invoked + * @param fail if true, treat + * @param waitForCompletion if non-null, a CountDownLatch that should be awaited on before + * invoking the callback. A (generous) timeout is still imposed to + * ensure tests complete. + * @return the captured set of ByteBuffer key-value pairs passed to the storage layer + */ + private void expectStore(final Callback callback, + final boolean fail, + final CountDownLatch waitForCompletion) { + EasyMock.expect(keyConverter.fromCopycatData(NAMESPACE, null, OFFSET_KEY_WRAPPED)).andReturn(OFFSET_KEY_SERIALIZED); + EasyMock.expect(valueConverter.fromCopycatData(NAMESPACE, null, OFFSET_VALUE)).andReturn(OFFSET_VALUE_SERIALIZED); + + final Capture> storeCallback = Capture.newInstance(); + EasyMock.expect(store.set(EasyMock.eq(OFFSETS_SERIALIZED), EasyMock.capture(storeCallback))) + .andAnswer(new IAnswer>() { + @Override + public Future answer() throws Throwable { + return service.submit(new Callable() { + @Override + public Void call() throws Exception { + if (waitForCompletion != null) + assertTrue(waitForCompletion.await(10000, TimeUnit.MILLISECONDS)); + + if (fail) { + storeCallback.getValue().onCompletion(exception, null); + } else { + storeCallback.getValue().onCompletion(null, null); + } + return null; + } + }); + } + }); + if (callback != null) { + if (fail) { + callback.onCompletion(EasyMock.eq(exception), EasyMock.eq((Void) null)); + } else { + callback.onCompletion(null, null); + } + } + PowerMock.expectLastCall(); + } + +} diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/ByteArrayProducerRecordEquals.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/ByteArrayProducerRecordEquals.java new file mode 100644 index 0000000000000..929ea859bb14b --- /dev/null +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/ByteArrayProducerRecordEquals.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.copycat.util; + +import org.apache.kafka.clients.producer.ProducerRecord; +import org.easymock.EasyMock; +import org.easymock.IArgumentMatcher; + +import java.util.Arrays; + +public class ByteArrayProducerRecordEquals implements IArgumentMatcher { + private ProducerRecord record; + + public static ProducerRecord eqProducerRecord(ProducerRecord in) { + EasyMock.reportMatcher(new ByteArrayProducerRecordEquals(in)); + return null; + } + + public ByteArrayProducerRecordEquals(ProducerRecord record) { + this.record = record; + } + + @Override + public boolean matches(Object argument) { + if (!(argument instanceof ProducerRecord)) + return false; + ProducerRecord other = (ProducerRecord) argument; + return record.topic().equals(other.topic()) && + record.partition() != null ? record.partition().equals(other.partition()) : other.partition() == null && + record.key() != null ? Arrays.equals(record.key(), other.key()) : other.key() == null && + record.value() != null ? Arrays.equals(record.value(), other.value()) : other.value() == null; + } + + @Override + public void appendTo(StringBuffer buffer) { + buffer.append(record.toString()); + } +} diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/MockTime.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/MockTime.java new file mode 100644 index 0000000000000..53149db016f84 --- /dev/null +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/MockTime.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.copycat.util; + +import org.apache.kafka.common.utils.Time; + +import java.util.concurrent.TimeUnit; + +/** + * A clock that you can manually advance by calling sleep + */ +public class MockTime implements Time { + + private long nanos = 0; + + public MockTime() { + this.nanos = System.nanoTime(); + } + + @Override + public long milliseconds() { + return TimeUnit.MILLISECONDS.convert(this.nanos, TimeUnit.NANOSECONDS); + } + + @Override + public long nanoseconds() { + return nanos; + } + + @Override + public void sleep(long ms) { + this.nanos += TimeUnit.NANOSECONDS.convert(ms, TimeUnit.MILLISECONDS); + } + +} diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/ShutdownableThreadTest.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/ShutdownableThreadTest.java new file mode 100644 index 0000000000000..4880ca1785c1d --- /dev/null +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/ShutdownableThreadTest.java @@ -0,0 +1,72 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

            + * http://www.apache.org/licenses/LICENSE-2.0 + *

            + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.copycat.util; + +import org.junit.Test; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class ShutdownableThreadTest { + + @Test + public void testGracefulShutdown() throws InterruptedException { + ShutdownableThread thread = new ShutdownableThread("graceful") { + @Override + public void execute() { + while (getRunning()) { + try { + Thread.sleep(1); + } catch (InterruptedException e) { + // Ignore + } + } + } + }; + thread.start(); + Thread.sleep(10); + assertTrue(thread.gracefulShutdown(1000, TimeUnit.MILLISECONDS)); + } + + @Test + public void testForcibleShutdown() throws InterruptedException { + final CountDownLatch startedLatch = new CountDownLatch(1); + ShutdownableThread thread = new ShutdownableThread("forcible") { + @Override + public void execute() { + try { + startedLatch.countDown(); + Thread.sleep(100000); + } catch (InterruptedException e) { + // Ignore + } + } + }; + thread.start(); + startedLatch.await(); + thread.forceShutdown(); + // Not all threads can be forcibly stopped since interrupt() doesn't work on threads in + // certain conditions, but in this case we know the thread is interruptible so we should be + // able join() it + thread.join(1000); + assertFalse(thread.isAlive()); + } +} diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/TestBackgroundThreadExceptionHandler.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/TestBackgroundThreadExceptionHandler.java new file mode 100644 index 0000000000000..5dc6d33041ebc --- /dev/null +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/TestBackgroundThreadExceptionHandler.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.copycat.util; + +/** + * An UncaughtExceptionHandler that can be registered with one or more threads which tracks the + * first exception so the main thread can check for uncaught exceptions. + */ +public class TestBackgroundThreadExceptionHandler implements Thread.UncaughtExceptionHandler { + private Throwable firstException = null; + + @Override + public void uncaughtException(Thread t, Throwable e) { + if (this.firstException == null) + this.firstException = e; + } + + public void verifyNoExceptions() { + if (this.firstException != null) + throw new AssertionError(this.firstException); + } +} diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/TestFuture.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/TestFuture.java new file mode 100644 index 0000000000000..8143c44a6148f --- /dev/null +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/TestFuture.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.copycat.util; + +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +public class TestFuture implements Future { + private volatile boolean resolved; + private T result; + private Throwable exception; + + public TestFuture() { + resolved = false; + } + + public void resolve(T val) { + this.result = val; + resolved = true; + + } + + public void resolve(Throwable t) { + exception = t; + resolved = true; + } + + @Override + public boolean cancel(boolean mayInterruptIfRunning) { + return false; + } + + @Override + public boolean isCancelled() { + return false; + } + + @Override + public boolean isDone() { + return resolved; + } + + @Override + public T get() throws InterruptedException, ExecutionException { + while (true) { + try { + return get(Integer.MAX_VALUE, TimeUnit.DAYS); + } catch (TimeoutException e) { + // ignore + } + } + } + + @Override + public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { + while (!resolved) { + this.wait(TimeUnit.MILLISECONDS.convert(timeout, unit)); + } + + if (exception != null) + throw new ExecutionException(exception); + return result; + } +} diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/ThreadedTest.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/ThreadedTest.java new file mode 100644 index 0000000000000..ed99247d95756 --- /dev/null +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/ThreadedTest.java @@ -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. + **/ + +package org.apache.kafka.copycat.util; + +import org.junit.After; +import org.junit.Before; + +/** + * Base class for tests that use threads. It sets up uncaught exception handlers for all known + * thread classes and checks for errors at the end of the test so that failures in background + * threads will cause the test to fail. + */ +public class ThreadedTest { + + protected TestBackgroundThreadExceptionHandler backgroundThreadExceptionHandler; + + @Before + public void setup() { + backgroundThreadExceptionHandler = new TestBackgroundThreadExceptionHandler(); + ShutdownableThread.funcaughtExceptionHandler = backgroundThreadExceptionHandler; + } + + @After + public void teardown() { + backgroundThreadExceptionHandler.verifyNoExceptions(); + ShutdownableThread.funcaughtExceptionHandler = null; + } +} diff --git a/copycat/runtime/src/test/resources/log4j.properties b/copycat/runtime/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..d5e90fe788f76 --- /dev/null +++ b/copycat/runtime/src/test/resources/log4j.properties @@ -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. +## +log4j.rootLogger=OFF, 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:%L)%n + +log4j.logger.org.apache.kafka=ERROR 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 2e94fee2fa35a..6af7b8068073c --- a/core/src/main/scala/kafka/Kafka.scala +++ b/core/src/main/scala/kafka/Kafka.scala @@ -17,23 +17,46 @@ 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 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 @@ -47,7 +70,9 @@ object Kafka extends Logging { 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/AclCommand.scala b/core/src/main/scala/kafka/admin/AclCommand.scala new file mode 100644 index 0000000000000..6a8a8a2f4c8cc --- /dev/null +++ b/core/src/main/scala/kafka/admin/AclCommand.scala @@ -0,0 +1,350 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.admin + +import joptsimple._ +import kafka.security.auth._ +import kafka.utils._ +import org.apache.kafka.common.security.auth.KafkaPrincipal +import org.apache.kafka.common.utils.Utils + +import scala.collection.JavaConverters._ + +object AclCommand { + + val Delimiter = ',' + val Newline = scala.util.Properties.lineSeparator + val ResourceTypeToValidOperations = Map[ResourceType, Set[Operation]] ( + Topic -> Set(Read, Write, Describe), + ConsumerGroup -> Set(Read), + Cluster -> Set(Create, ClusterAction) + ) + + def main(args: Array[String]) { + + val opts = new AclCommandOptions(args) + + if (opts.options.has(opts.helpOpt)) + CommandLineUtils.printUsageAndDie(opts.parser, "Usage:") + + opts.checkArgs() + + var authorizerProperties = Map.empty[String, Any] + if (opts.options.has(opts.authorizerPropertiesOpt)) { + val props = opts.options.valuesOf(opts.authorizerPropertiesOpt).asScala.map(_.split("=")) + props.foreach(pair => authorizerProperties += (pair(0).trim -> pair(1).trim)) + } + + val authorizerClass = opts.options.valueOf(opts.authorizerOpt) + val authZ: Authorizer = CoreUtils.createObject(authorizerClass) + authZ.configure(authorizerProperties.asJava) + + try { + if (opts.options.has(opts.addOpt)) + addAcl(authZ, opts) + else if (opts.options.has(opts.removeOpt)) + removeAcl(authZ, opts) + else if (opts.options.has(opts.listOpt)) + listAcl(authZ, opts) + } catch { + case e: Throwable => + println(s"Error while executing topic Acl command ${e.getMessage}") + println(Utils.stackTrace(e)) + System.exit(-1) + } + } + + private def addAcl(authZ: Authorizer, opts: AclCommandOptions) { + val resourceToAcl = getResourceToAcls(opts) + + if (resourceToAcl.values.exists(_.isEmpty)) + CommandLineUtils.printUsageAndDie(opts.parser, "You must specify one of: --allow-principals, --deny-principals when trying to add acls.") + + for ((resource, acls) <- resourceToAcl) { + val acls = resourceToAcl(resource) + println(s"Adding following acls for resource: $resource $Newline ${acls.map("\t" + _).mkString(Newline)} $Newline") + authZ.addAcls(acls, resource) + } + + listAcl(authZ, opts) + } + + private def removeAcl(authZ: Authorizer, opts: AclCommandOptions) { + val resourceToAcl = getResourceToAcls(opts) + + for ((resource, acls) <- resourceToAcl) { + if (acls.isEmpty) { + if (confirmAction(s"Are you sure you want to delete all acls for resource: $resource y/n?")) + authZ.removeAcls(resource) + } else { + if (confirmAction(s"Are you sure you want to remove acls: $Newline ${acls.map("\t" + _).mkString(Newline)} $Newline from resource $resource y/n?")) + authZ.removeAcls(acls, resource) + } + } + + listAcl(authZ, opts) + } + + private def listAcl(authZ: Authorizer, opts: AclCommandOptions) { + val resources = getResource(opts, dieIfNoResourceFound = false) + + val resourceToAcls = if(resources.isEmpty) + authZ.getAcls() + else + resources.map(resource => (resource -> authZ.getAcls(resource))) + + for ((resource, acls) <- resourceToAcls) + println(s"Following is list of acls for resource: $resource $Newline ${acls.map("\t" + _).mkString(Newline)} $Newline") + } + + private def getResourceToAcls(opts: AclCommandOptions): Map[Resource, Set[Acl]] = { + var resourceToAcls = Map.empty[Resource, Set[Acl]] + + //if none of the --producer or --consumer options are specified , just construct acls from CLI options. + if (!opts.options.has(opts.producerOpt) && !opts.options.has(opts.consumerOpt)) { + resourceToAcls ++= getCliResourceToAcls(opts) + } + + //users are allowed to specify both --producer and --consumer options in a single command. + if (opts.options.has(opts.producerOpt)) + resourceToAcls ++= getProducerResourceToAcls(opts) + + if (opts.options.has(opts.consumerOpt)) + resourceToAcls ++= getConsumerResourceToAcls(opts).map { case (k, v) => k -> (v ++ resourceToAcls.getOrElse(k, Set.empty[Acl])) } + + validateOperation(opts, resourceToAcls) + + resourceToAcls + } + + private def getProducerResourceToAcls(opts: AclCommandOptions): Map[Resource, Set[Acl]] = { + val topics: Set[Resource] = getResource(opts).filter(_.resourceType == Topic) + + val acls = getAcl(opts, Set(Write, Describe)) + + //Write, Describe permission on topics, Create permission on cluster + topics.map(_ -> acls).toMap[Resource, Set[Acl]] + + (Resource.ClusterResource -> getAcl(opts, Set(Create))) + } + + private def getConsumerResourceToAcls(opts: AclCommandOptions): Map[Resource, Set[Acl]] = { + val resources = getResource(opts) + + val topics: Set[Resource] = getResource(opts).filter(_.resourceType == Topic) + val consumerGroups: Set[Resource] = resources.filter(_.resourceType == ConsumerGroup) + + //Read,Describe on topic, Read on consumerGroup + Create on cluster + + val acls = getAcl(opts, Set(Read, Describe)) + + topics.map(_ -> acls).toMap[Resource, Set[Acl]] ++ + consumerGroups.map(_ -> getAcl(opts, Set(Read))).toMap[Resource, Set[Acl]] + } + + private def getCliResourceToAcls(opts: AclCommandOptions): Map[Resource, Set[Acl]] = { + val acls = getAcl(opts) + val resources = getResource(opts) + resources.map(_ -> acls).toMap + } + + private def getAcl(opts: AclCommandOptions, operations: Set[Operation]): Set[Acl] = { + val allowedPrincipals = getPrincipals(opts, opts.allowPrincipalsOpt) + + val deniedPrincipals = getPrincipals(opts, opts.denyPrincipalsOpt) + + val allowedHosts = getHosts(opts, opts.allowHostsOpt, opts.allowPrincipalsOpt) + + val deniedHosts = getHosts(opts, opts.denyHostssOpt, opts.denyPrincipalsOpt) + + val acls = new collection.mutable.HashSet[Acl] + if (allowedHosts.nonEmpty && allowedPrincipals.nonEmpty) + acls ++= getAcls(allowedPrincipals, Allow, operations, allowedHosts) + + if (deniedHosts.nonEmpty && deniedPrincipals.nonEmpty) + acls ++= getAcls(deniedPrincipals, Deny, operations, deniedHosts) + + acls.toSet + } + + private def getAcl(opts: AclCommandOptions): Set[Acl] = { + val operations = opts.options.valuesOf(opts.operationsOpt).asScala.map(operation => Operation.fromString(operation.trim)).toSet + getAcl(opts, operations) + } + + def getAcls(principals: Set[KafkaPrincipal], permissionType: PermissionType, operations: Set[Operation], + hosts: Set[String]): Set[Acl] = { + for { + principal <- principals + operation <- operations + host <- hosts + } yield new Acl(principal, permissionType, host, operation) + } + + private def getHosts(opts: AclCommandOptions, hostOptionSpec: ArgumentAcceptingOptionSpec[String], + principalOptionSpec: ArgumentAcceptingOptionSpec[String]): Set[String] = { + if (opts.options.has(hostOptionSpec)) + opts.options.valuesOf(hostOptionSpec).asScala.map(_.trim).toSet + else if (opts.options.has(principalOptionSpec)) + Set[String](Acl.WildCardHost) + else + Set.empty[String] + } + + private def getPrincipals(opts: AclCommandOptions, principalOptionSpec: ArgumentAcceptingOptionSpec[String]): Set[KafkaPrincipal] = { + if (opts.options.has(principalOptionSpec)) + opts.options.valuesOf(principalOptionSpec).asScala.map(s => KafkaPrincipal.fromString(s.trim)).toSet + else + Set.empty[KafkaPrincipal] + } + + private def getResource(opts: AclCommandOptions, dieIfNoResourceFound: Boolean = true): Set[Resource] = { + var resources = Set.empty[Resource] + if (opts.options.has(opts.topicOpt)) + opts.options.valuesOf(opts.topicOpt).asScala.foreach(topic => resources += new Resource(Topic, topic.trim)) + + if (opts.options.has(opts.clusterOpt)) + resources += Resource.ClusterResource + + if (opts.options.has(opts.groupOpt)) + opts.options.valuesOf(opts.groupOpt).asScala.foreach(consumerGroup => resources += new Resource(ConsumerGroup, consumerGroup.trim)) + + if (resources.isEmpty && dieIfNoResourceFound) + CommandLineUtils.printUsageAndDie(opts.parser, "You must provide at least one resource: --topic or --cluster or --consumer-group ") + + resources + } + + private def confirmAction(msg: String): Boolean = { + println(msg) + Console.readLine().equalsIgnoreCase("y") + } + + private def validateOperation(opts: AclCommandOptions, resourceToAcls: Map[Resource, Set[Acl]]) = { + for ((resource, acls) <- resourceToAcls) { + val validOps = ResourceTypeToValidOperations(resource.resourceType) + if ((acls.map(_.operation) -- validOps).nonEmpty) + CommandLineUtils.printUsageAndDie(opts.parser, s"ResourceType ${resource.resourceType} only supports operations ${validOps.mkString(Delimiter.toString)}") + } + } + + class AclCommandOptions(args: Array[String]) { + val parser = new OptionParser + val authorizerOpt = parser.accepts("authorizer", "Fully qualified class name of the authorizer, defaults to kafka.security.auth.SimpleAclAuthorizer.") + .withRequiredArg + .describedAs("authorizer") + .ofType(classOf[String]) + .defaultsTo(classOf[SimpleAclAuthorizer].getName) + + val authorizerPropertiesOpt = parser.accepts("authorizer-properties", "REQUIRED: properties required to configure an instance of Authorizer. " + + "These are comma separated key=val pairs. For the default authorizer the example values are: " + + "zookeeper.connect=localhost:2181") + .withRequiredArg + .describedAs("authorizer-properties") + .ofType(classOf[String]) + .withValuesSeparatedBy(Delimiter) + + val topicOpt = parser.accepts("topic", "Comma separated list of topic to which acls should be added or removed. " + + "A value of * indicates acl should apply to all topics.") + .withRequiredArg + .describedAs("topic") + .ofType(classOf[String]) + .withValuesSeparatedBy(Delimiter) + + val clusterOpt = parser.accepts("cluster", "Add/Remove cluster acls.") + val groupOpt = parser.accepts("consumer-group", "Comma separated list of consumer groups to which the acls should be added or removed. " + + "A value of * indicates the acls should apply to all consumer-groups.") + .withRequiredArg + .describedAs("consumer-group") + .ofType(classOf[String]) + .withValuesSeparatedBy(Delimiter) + + val addOpt = parser.accepts("add", "Indicates you are trying to add acls.") + val removeOpt = parser.accepts("remove", "Indicates you are trying to remove acls.") + val listOpt = parser.accepts("list", "List acls for the specified resource, use --topic or --consumer-group or --cluster to specify a resource.") + + val operationsOpt = parser.accepts("operations", "Comma separated list of operations, default is All. Valid operation names are: " + Newline + + Operation.values.map("\t" + _).mkString(Newline) + Newline) + .withRequiredArg + .ofType(classOf[String]) + .defaultsTo(All.name) + .withValuesSeparatedBy(Delimiter) + + val allowPrincipalsOpt = parser.accepts("allow-principals", "Comma separated list of principals where principal is in principalType:name format." + + " User:* is the wild card indicating all users.") + .withRequiredArg + .describedAs("allow-principals") + .ofType(classOf[String]) + .withValuesSeparatedBy(Delimiter) + + val denyPrincipalsOpt = parser.accepts("deny-principals", "Comma separated list of principals where principal is in " + + "principalType: name format. By default anyone not in --allow-principals list is denied access. " + + "You only need to use this option as negation to already allowed set. " + + "For example if you wanted to allow access to all users in the system but not test-user you can define an acl that " + + "allows access to User:* and specify --deny-principals=User:test@EXAMPLE.COM. " + + "AND PLEASE REMEMBER DENY RULES TAKES PRECEDENCE OVER ALLOW RULES.") + .withRequiredArg + .describedAs("deny-principals") + .ofType(classOf[String]) + .withValuesSeparatedBy(Delimiter) + + val allowHostsOpt = parser.accepts("allow-hosts", "Comma separated list of hosts from which principals listed in --allow-principals will have access. " + + "If you have specified --allow-principals then the default for this option will be set to * which allows access from all hosts.") + .withRequiredArg + .describedAs("allow-hosts") + .ofType(classOf[String]) + .withValuesSeparatedBy(Delimiter) + + val denyHostssOpt = parser.accepts("deny-hosts", "Comma separated list of hosts from which principals listed in --deny-principals will be denied access. " + + "If you have specified --deny-principals then the default for this option will be set to * which denies access from all hosts.") + .withRequiredArg + .describedAs("deny-hosts") + .ofType(classOf[String]) + .withValuesSeparatedBy(Delimiter) + + val producerOpt = parser.accepts("producer", "Convenience option to add/remove acls for producer role. " + + "This will generate acls that allows WRITE,DESCRIBE on topic and CREATE on cluster. ") + + val consumerOpt = parser.accepts("consumer", "Convenience option to add/remove acls for consumer role. " + + "This will generate acls that allows READ,DESCRIBE on topic and READ on consumer-group.") + + val helpOpt = parser.accepts("help", "Print usage information.") + + val options = parser.parse(args: _*) + + def checkArgs() { + CommandLineUtils.checkRequiredArgs(parser, options, authorizerPropertiesOpt) + + val actions = Seq(addOpt, removeOpt, listOpt).count(options.has) + if (actions != 1) + CommandLineUtils.printUsageAndDie(parser, "Command must include exactly one action: --list, --add, --remove. ") + + CommandLineUtils.checkInvalidArgs(parser, options, listOpt, Set(producerOpt, consumerOpt, allowHostsOpt, allowPrincipalsOpt, denyHostssOpt, denyPrincipalsOpt)) + + //when --producer or --consumer is specified , user should not specify operations as they are inferred and we also disallow --deny-principals and --deny-hosts. + CommandLineUtils.checkInvalidArgs(parser, options, producerOpt, Set(operationsOpt, denyPrincipalsOpt, denyHostssOpt)) + CommandLineUtils.checkInvalidArgs(parser, options, consumerOpt, Set(operationsOpt, denyPrincipalsOpt, denyHostssOpt)) + + if (options.has(producerOpt) && !options.has(topicOpt)) + CommandLineUtils.printUsageAndDie(parser, "With --producer you must specify a --topic") + + if (options.has(consumerOpt) && (!options.has(topicOpt) || !options.has(groupOpt) || (!options.has(producerOpt) && options.has(clusterOpt)))) + CommandLineUtils.printUsageAndDie(parser, "With --consumer you must specify a --topic and a --consumer-group and no --cluster option should be specified.") + } + } + +} diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index 94c53320b768b..9966660cf668f 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -17,26 +17,32 @@ package kafka.admin +import kafka.common._ +import kafka.cluster.{BrokerEndPoint, Broker} + +import kafka.log.LogConfig +import kafka.server.ConfigType +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 TopicConfigChangeZnodePrefix = "config_change_" + val AdminClientId = "__admin_client" + val EntityConfigChangeZnodePrefix = "config_change_" /** * There are 2 goals of replica assignment: @@ -96,14 +102,12 @@ object AdminUtils extends Logging { * @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) { + checkBrokerAvailable: Boolean = true) { val existingPartitionsReplicaList = ZkUtils.getReplicaAssignmentForTopics(zkClient, List(topic)) if (existingPartitionsReplicaList.size == 0) throw new AdminOperationException("The topic %s does not exist".format(topic)) @@ -130,7 +134,7 @@ 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, config, true) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaList, update = true) } def getManualReplicaAssignment(replicaAssignmentList: String, availableBrokerList: Set[Int], startPartitionId: Int, checkBrokerAvailable: Boolean = true): Map[Int, List[Int]] = { @@ -156,9 +160,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)) @@ -171,7 +235,7 @@ object AdminUtils extends Logging { val replicaAssignment = AdminUtils.assignReplicasToBrokers(brokerList, partitions, replicationFactor) AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, replicaAssignment, topicConfig) } - + def createOrUpdateTopicPartitionAssignmentPathInZK(zkClient: ZkClient, topic: String, partitionReplicaAssignment: Map[Int, Seq[Int]], @@ -179,21 +243,35 @@ object AdminUtils extends Logging { update: Boolean = false) { // validate arguments Topic.validate(topic) - LogConfig.validate(config) require(partitionReplicaAssignment.values.map(_.size).toSet.size == 1, "All partitions should have the same number of replicas.") val topicPath = ZkUtils.getTopicPath(topic) - if(!update && zkClient.exists(topicPath)) - throw new TopicExistsException("Topic \"%s\" already exists.".format(topic)) + + if (!update) { + if (zkClient.exists(topicPath)) + throw new TopicExistsException("Topic \"%s\" already exists.".format(topic)) + else if (Topic.hasCollisionChars(topic)) { + val allTopics = ZkUtils.getAllTopics(zkClient) + val collidingTopics = allTopics.filter(t => Topic.hasCollision(topic, t)) + if (collidingTopics.nonEmpty) { + throw new InvalidTopicException("Topic \"%s\" collides with existing topics: %s".format(topic, collidingTopics.mkString(", "))) + } + } + } + partitionReplicaAssignment.values.foreach(reps => require(reps.size == reps.toSet.size, "Duplicate replica assignment found: " + partitionReplicaAssignment)) - - // write out the config if there is any, this isn't transactional with the partition assignments - writeTopicConfig(zkClient, topic, config) - + + // Configs only matter if a topic is being created. Changing configs via AlterTopic is not supported + if (!update) { + // write out the config if there is any, this isn't transactional with the partition assignments + LogConfig.validate(config) + writeEntityConfig(zkClient, ConfigType.Topic, topic, config) + } + // create the partition assignment writeTopicPartitionAssignment(zkClient, topic, partitionReplicaAssignment, update) } - + private def writeTopicPartitionAssignment(zkClient: ZkClient, topic: String, replicaAssignment: Map[Int, Seq[Int]], update: Boolean) { try { val zkPath = ZkUtils.getTopicPath(topic) @@ -212,7 +290,19 @@ object AdminUtils extends Logging { case e2: Throwable => throw new AdminOperationException(e2.toString) } } - + + /** + * Update the config for a client and create a change notification so the change will propagate to other brokers + * @param zkClient: The ZkClient handle used to write the new config to zookeeper + * @param clientId: The clientId for which configs are being changed + * @param configs: The final set of configs that will be applied to the topic. If any new configs need to be added or + * existing configs need to be deleted, it should be done prior to invoking this API + * + */ + def changeClientIdConfig(zkClient: ZkClient, clientId: String, configs: Properties) { + changeEntityConfig(zkClient, ConfigType.Client, clientId, configs) + } + /** * Update the config for an existing topic and create a change notification so the change will propagate to other brokers * @param zkClient: The ZkClient handle used to write the new config to zookeeper @@ -224,55 +314,69 @@ object AdminUtils extends Logging { def changeTopicConfig(zkClient: ZkClient, topic: String, configs: Properties) { if(!topicExists(zkClient, topic)) throw new AdminOperationException("Topic \"%s\" does not exist.".format(topic)) - // remove the topic overrides LogConfig.validate(configs) + changeEntityConfig(zkClient, ConfigType.Topic, topic, configs) + } + private def changeEntityConfig(zkClient: ZkClient, entityType: String, entityName: String, configs: Properties) { // write the new config--may not exist if there were previously no overrides - writeTopicConfig(zkClient, topic, configs) - + writeEntityConfig(zkClient, entityType, entityName, configs) + // create the change notification - zkClient.createPersistentSequential(ZkUtils.TopicConfigChangesPath + "/" + TopicConfigChangeZnodePrefix, Json.encode(topic)) + val seqNode = ZkUtils.EntityConfigChangesPath + "/" + EntityConfigChangeZnodePrefix + val content = Json.encode(getConfigChangeZnodeData(entityType, entityName)) + zkClient.createPersistentSequential(seqNode, content) } - + + def getConfigChangeZnodeData(entityType: String, entityName: String) : Map[String, Any] = { + Map("version" -> 1, "entity_type" -> entityType, "entity_name" -> entityName) + } + /** * Write out the topic config to zk, if there is any */ - private def writeTopicConfig(zkClient: ZkClient, topic: String, config: Properties) { + private def writeEntityConfig(zkClient: ZkClient, entityType: String, entityName: String, config: Properties) { val configMap: mutable.Map[String, String] = { import JavaConversions._ config } val map = Map("version" -> 1, "config" -> configMap) - ZkUtils.updatePersistentPath(zkClient, ZkUtils.getTopicConfigPath(topic), Json.encode(map)) + ZkUtils.updatePersistentPath(zkClient, ZkUtils.getEntityConfigPath(entityType, entityName), Json.encode(map)) } /** - * Read the topic config (if any) from zk + * Read the entity (topic or client) config (if any) from zk */ - def fetchTopicConfig(zkClient: ZkClient, topic: String): Properties = { - val str: String = zkClient.readData(ZkUtils.getTopicConfigPath(topic), true) + def fetchEntityConfig(zkClient: ZkClient, entityType: String, entity: String): Properties = { + val str: String = zkClient.readData(ZkUtils.getEntityConfigPath(entityType, entity), true) val props = new Properties() 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 _ => throw new IllegalArgumentException("Invalid topic config: " + str) + case Some(config: Map[_, _]) => + for(configTup <- config) + configTup match { + case (k: String, v: String) => + props.setProperty(k, v) + case _ => throw new IllegalArgumentException("Invalid " + entityType + " config: " + str) + } + case _ => throw new IllegalArgumentException("Invalid " + entityType + " config: " + str) } - case o => throw new IllegalArgumentException("Unexpected value in config: " + str) + case o => throw new IllegalArgumentException("Unexpected value in config:(%s), entity_type: (%s), entity: (%s)" + .format(str, entityType, entity)) } } props } def fetchAllTopicConfigs(zkClient: ZkClient): Map[String, Properties] = - ZkUtils.getAllTopics(zkClient).map(topic => (topic, fetchTopicConfig(zkClient, topic))).toMap + ZkUtils.getAllTopics(zkClient).map(topic => (topic, fetchEntityConfig(zkClient, ConfigType.Topic, topic))).toMap def fetchTopicMetadataFromZk(topic: String, zkClient: ZkClient): TopicMetadata = fetchTopicMetadataFromZk(topic, zkClient, new mutable.HashMap[Int, Broker]) @@ -282,7 +386,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) @@ -293,22 +399,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/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala new file mode 100644 index 0000000000000..17bbbcbd8f38f --- /dev/null +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala @@ -0,0 +1,174 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.admin + +import joptsimple._ +import java.util.Properties +import kafka.log.LogConfig +import kafka.server.ConfigType +import kafka.utils.{ZkUtils, CommandLineUtils} +import org.I0Itec.zkclient.ZkClient +import scala.collection._ +import scala.collection.JavaConversions._ +import org.apache.kafka.common.utils.Utils + + +/** + * This script can be used to change configs for topics/clients dynamically + */ +object ConfigCommand { + + def main(args: Array[String]): Unit = { + + val opts = new ConfigCommandOptions(args) + + if(args.length == 0) + CommandLineUtils.printUsageAndDie(opts.parser, "Add/Remove entity (topics/clients) configs") + + opts.checkArgs() + + val zkClient = ZkUtils.createZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000) + + try { + if (opts.options.has(opts.alterOpt)) + alterConfig(zkClient, opts) + else if (opts.options.has(opts.describeOpt)) + describeConfig(zkClient, opts) + } catch { + case e: Throwable => + println("Error while executing topic command " + e.getMessage) + println(Utils.stackTrace(e)) + } finally { + zkClient.close() + } + } + + private def alterConfig(zkClient: ZkClient, opts: ConfigCommandOptions) { + val configsToBeAdded = parseConfigsToBeAdded(opts) + val configsToBeDeleted = parseConfigsToBeDeleted(opts) + val entityType = opts.options.valueOf(opts.entityType) + val entityName = opts.options.valueOf(opts.entityName) + + // compile the final set of configs + val configs = AdminUtils.fetchEntityConfig(zkClient, entityType, entityName) + configs.putAll(configsToBeAdded) + configsToBeDeleted.foreach(config => configs.remove(config)) + + if (entityType.equals(ConfigType.Topic)) { + AdminUtils.changeTopicConfig(zkClient, entityName, configs) + println("Updated config for topic: \"%s\".".format(entityName)) + } else { + AdminUtils.changeClientIdConfig(zkClient, entityName, configs) + println("Updated config for clientId: \"%s\".".format(entityName)) + } + } + + private def describeConfig(zkClient: ZkClient, opts: ConfigCommandOptions) { + val entityType = opts.options.valueOf(opts.entityType) + val entityNames: Seq[String] = + if (opts.options.has(opts.entityName)) + Seq(opts.options.valueOf(opts.entityName)) + else + ZkUtils.getAllEntitiesWithConfig(zkClient, entityType) + + for (entityName <- entityNames) { + val configs = AdminUtils.fetchEntityConfig(zkClient, entityType, entityName) + println("Configs for %s:%s are %s" + .format(entityType, entityName, configs.map(kv => kv._1 + "=" + kv._2).mkString(","))) + } + } + + private[admin] def parseConfigsToBeAdded(opts: ConfigCommandOptions): Properties = { + val configsToBeAdded = opts.options.valuesOf(opts.addedConfig).map(_.split("""\s*=\s*""")) + require(configsToBeAdded.forall(config => config.length == 2), + "Invalid entity 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[admin] def parseConfigsToBeDeleted(opts: ConfigCommandOptions): Seq[String] = { + if (opts.options.has(opts.deletedConfig)) { + val configsToBeDeleted = opts.options.valuesOf(opts.deletedConfig).map(_.trim()) + val propsToBeDeleted = new Properties + configsToBeDeleted.foreach(propsToBeDeleted.setProperty(_, "")) + configsToBeDeleted + } + else + Seq.empty + } + + class ConfigCommandOptions(args: Array[String]) { + val parser = new OptionParser + val zkConnectOpt = parser.accepts("zookeeper", "REQUIRED: The connection string for the zookeeper connection in the form host:port. " + + "Multiple URLS can be given to allow fail-over.") + .withRequiredArg + .describedAs("urls") + .ofType(classOf[String]) + val alterOpt = parser.accepts("alter", "Alter the configuration for the entity.") + val describeOpt = parser.accepts("describe", "List configs for the given entity.") + val entityType = parser.accepts("entity-type", "Type of entity (topics/clients)") + .withRequiredArg + .ofType(classOf[String]) + val entityName = parser.accepts("entity-name", "Name of entity (topic name/client id)") + .withRequiredArg + .ofType(classOf[String]) + + val nl = System.getProperty("line.separator") + val addedConfig = parser.accepts("added-config", "Key Value pairs configs to add 'k1=v1,k2=v2'. The following is a list of valid configurations: " + + "For entity_type '" + ConfigType.Topic + "': " + nl + LogConfig.configNames.map("\t" + _).mkString(nl) + nl + + "For entity_type '" + ConfigType.Client + "' currently no configs are processed by the brokers") + .withRequiredArg + .ofType(classOf[String]) + .withValuesSeparatedBy(',') + val deletedConfig = parser.accepts("deleted-config", "config keys to remove 'k1,k2'") + .withRequiredArg + .ofType(classOf[String]) + .withValuesSeparatedBy(',') + val helpOpt = parser.accepts("help", "Print usage information.") + val options = parser.parse(args : _*) + + val allOpts: Set[OptionSpec[_]] = Set(alterOpt, describeOpt, entityType, entityName, addedConfig, deletedConfig, helpOpt) + + def checkArgs() { + // should have exactly one action + val actions = Seq(alterOpt, describeOpt).count(options.has _) + if(actions != 1) + CommandLineUtils.printUsageAndDie(parser, "Command must include exactly one action: --describe, --alter") + + // check required args + CommandLineUtils.checkRequiredArgs(parser, options, zkConnectOpt, entityType) + CommandLineUtils.checkInvalidArgs(parser, options, alterOpt, Set(describeOpt)) + CommandLineUtils.checkInvalidArgs(parser, options, describeOpt, Set(alterOpt, addedConfig, deletedConfig)) + if(options.has(alterOpt)) { + if(! options.has(entityName)) + throw new IllegalArgumentException("--entity-name must be specified with --alter") + + val isAddedPresent: Boolean = options.has(addedConfig) + val isDeletedPresent: Boolean = options.has(deletedConfig) + if(! isAddedPresent && ! isDeletedPresent) + throw new IllegalArgumentException("At least one of --added-config or --deleted-config must be specified with --alter") + } + val entityTypeVal = options.valueOf(entityType) + if(! entityTypeVal.equals(ConfigType.Topic) && ! entityTypeVal.equals(ConfigType.Client)) { + throw new IllegalArgumentException("--entity-type must be '%s' or '%s'".format(ConfigType.Topic, ConfigType.Client)) + } + } + } + +} 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/PreferredReplicaLeaderElectionCommand.scala b/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala old mode 100644 new mode 100755 index c7918483c0204..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 { @@ -53,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) @@ -78,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 691d69a49a240..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 { @@ -37,7 +38,7 @@ object ReassignPartitionsCommand extends Logging { 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) @@ -81,8 +82,14 @@ object ReassignPartitionsCommand extends Logging { 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]]() @@ -103,17 +110,29 @@ object ReassignPartitionsCommand extends Logging { 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)) } @@ -189,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/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala old mode 100644 new mode 100755 index 3b2166aa4e2e8..f1405a5b2961b --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -19,19 +19,21 @@ package kafka.admin import joptsimple._ import java.util.Properties +import kafka.common.{Topic, AdminCommandFailedException} +import kafka.utils.CommandLineUtils 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 kafka.server.OffsetManager -import org.apache.kafka.common.utils.Utils.formatAddress +import kafka.server.{ConfigType, OffsetManager} +import org.apache.kafka.common.utils.Utils +import kafka.coordinator.ConsumerCoordinator -object TopicCommand { +object TopicCommand extends Logging { def main(args: Array[String]): Unit = { @@ -47,8 +49,8 @@ object TopicCommand { 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) @@ -62,11 +64,14 @@ object TopicCommand { deleteTopic(zkClient, opts) } catch { case e: Throwable => - println("Error while executing topic command " + e.getMessage) - println(Utils.stackTrace(e)) + 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] = { @@ -82,9 +87,11 @@ object TopicCommand { def createTopic(zkClient: ZkClient, opts: TopicCommandOptions) { val topic = opts.options.valueOf(opts.topicOpt) val configs = parseTopicConfigsToBeAdded(opts) + if (Topic.hasCollisionChars(topic)) + println("WARNING: Due to limitations in metric names, topics with a period ('.') or underscore ('_') could collide. To avoid issues it is best to use either, but not both.") if (opts.options.has(opts.replicaAssignmentOpt)) { val assignment = parseReplicaAssignment(opts.options.valueOf(opts.replicaAssignmentOpt)) - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, assignment, configs) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, assignment, configs, update = false) } else { CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, opts.partitionsOpt, opts.replicationFactorOpt) val partitions = opts.options.valueOf(opts.partitionsOpt).intValue @@ -96,26 +103,20 @@ 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 - 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 == OffsetManager.OffsetsTopicName) { + 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, config = configs) + AdminUtils.addPartitions(zkClient, topic, nPartitions, replicaAssignmentStr) println("Adding partitions succeeded!") } } @@ -135,19 +136,26 @@ object TopicCommand { def deleteTopic(zkClient: ZkClient, opts: TopicCommandOptions) { val topics = getTopics(zkClient, opts) if (topics.length == 0) { - println("Topic %s does not exist".format(opts.options.valueOf(opts.topicOpt))) + 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 { - 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.") + 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 e2: Throwable => + case e: AdminOperationException => + throw e + case e: Throwable => throw new AdminOperationException("Error while deleting topic %s".format(topic)) - } + } } } @@ -164,7 +172,7 @@ object TopicCommand { val describePartitions: Boolean = !reportOverriddenConfigs val sortedPartitions = topicPartitionAssignment.toList.sortWith((m1, m2) => m1._1 < m2._1) if (describeConfigs) { - val configs = AdminUtils.fetchTopicConfig(zkClient, topic) + val configs = AdminUtils.fetchEntityConfig(zkClient, ConfigType.Topic, topic) if (!reportOverriddenConfigs || configs.size() != 0) { val numPartitions = topicPartitionAssignment.size val replicationFactor = topicPartitionAssignment.head._2.size @@ -192,9 +200,7 @@ object TopicCommand { } } } - - def formatBroker(broker: Broker) = broker.id + " (" + formatAddress(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), @@ -205,23 +211,14 @@ object TopicCommand { props } - def parseTopicConfigsToBeDeleted(opts: TopicCommandOptions): Seq[String] = { - if (opts.options.has(opts.deleteConfigOpt)) { - val configsToBeDeleted = opts.options.valuesOf(opts.deleteConfigOpt).map(_.trim()) - val propsToBeDeleted = new Properties - configsToBeDeleted.foreach(propsToBeDeleted.setProperty(_, "")) - LogConfig.validateNames(propsToBeDeleted) - configsToBeDeleted - } - else - Seq.empty - } - def parseReplicaAssignment(replicaAssignmentList: String): Map[Int, List[Int]] = { val partitionList = replicaAssignmentList.split(",") val ret = new mutable.HashMap[Int, List[Int]]() for (i <- 0 until partitionList.size) { val brokerList = partitionList(i).split(":").map(s => s.trim().toInt) + 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) @@ -239,7 +236,7 @@ object TopicCommand { 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 alterOpt = parser.accepts("alter", "Alter the number of partitions and/or replica assignment for a 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 or describe. Can also accept a regular " + @@ -248,16 +245,12 @@ object TopicCommand { .describedAs("topic") .ofType(classOf[String]) 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 + + val configOpt = parser.accepts("config", "A configuration override for the topic being created." + + "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("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]) val partitionsOpt = parser.accepts("partitions", "The number of partitions for the topic being created or " + "altered (WARNING: If partitions are increased for a topic that has a key, the partition logic or ordering of the messages will be affected") .withRequiredArg @@ -291,11 +284,13 @@ object TopicCommand { // check invalid args CommandLineUtils.checkInvalidArgs(parser, options, configOpt, allTopicLevelOpts -- Set(alterOpt, createOpt)) - 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)) + // Topic configs cannot be changed with alterTopic + CommandLineUtils.checkInvalidArgs(parser, options, alterOpt, Set(configOpt)) + 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/ApiUtils.scala b/core/src/main/scala/kafka/api/ApiUtils.scala index 1f80de1638978..ca0a63f8423e2 100644 --- a/core/src/main/scala/kafka/api/ApiUtils.scala +++ b/core/src/main/scala/kafka/api/ApiUtils.scala @@ -14,10 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package kafka.api +package kafka.api import java.nio._ +import java.nio.channels.GatheringByteChannel import kafka.common._ +import org.apache.kafka.common.network.TransportLayer /** * Helper functions specific to parsing or serializing requests and responses @@ -107,5 +109,10 @@ object ApiUtils { throw new KafkaException(name + " has value " + value + " which is not in the range " + range + ".") else value } + + private[api] def hasPendingWrites(channel: GatheringByteChannel): Boolean = channel match { + case t: TransportLayer => t.hasPendingWrites + case _ => false + } } 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..c9c19761ec288 --- /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.9.0" -> KAFKA_090 + ) + + 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_090 extends ApiVersion { + val version: String = "0.9.0.X" + val id: Int = 3 +} diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala index 6d00ed090d76c..258d5fe289fbb 100644 --- a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala @@ -18,9 +18,10 @@ package kafka.api import java.nio.ByteBuffer -import kafka.network.{BoundedByteBufferSend, RequestChannel} -import kafka.network.RequestChannel.Response + import kafka.common.ErrorMapping +import kafka.network.{RequestOrResponseSend, RequestChannel} +import kafka.network.RequestChannel.Response object ConsumerMetadataRequest { val CurrentVersion = 0.shortValue @@ -63,8 +64,8 @@ case class ConsumerMetadataRequest(group: String, override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { // return ConsumerCoordinatorNotAvailable for all uncaught errors - val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, correlationId) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse))) } def describe(details: Boolean) = { diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala index 84f60178f6eba..ea1c0d04b1036 100644 --- a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala +++ b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala @@ -18,18 +18,18 @@ package kafka.api import java.nio.ByteBuffer -import kafka.cluster.Broker +import kafka.cluster.BrokerEndPoint import kafka.common.ErrorMapping object ConsumerMetadataResponse { val CurrentVersion = 0 - private val NoBrokerOpt = Some(Broker(id = -1, host = "", port = -1)) + private val NoBrokerEndpointOpt = Some(BrokerEndPoint(id = -1, host = "", port = -1)) def readFrom(buffer: ByteBuffer) = { val correlationId = buffer.getInt val errorCode = buffer.getShort - val broker = Broker.readFrom(buffer) + val broker = BrokerEndPoint.readFrom(buffer) val coordinatorOpt = if (errorCode == ErrorMapping.NoError) Some(broker) else @@ -40,18 +40,18 @@ object ConsumerMetadataResponse { } -case class ConsumerMetadataResponse (coordinatorOpt: Option[Broker], errorCode: Short, correlationId: Int = 0) +case class ConsumerMetadataResponse (coordinatorOpt: Option[BrokerEndPoint], errorCode: Short, correlationId: Int) extends RequestOrResponse() { def sizeInBytes = 4 + /* correlationId */ 2 + /* error code */ - coordinatorOpt.orElse(ConsumerMetadataResponse.NoBrokerOpt).get.sizeInBytes + coordinatorOpt.orElse(ConsumerMetadataResponse.NoBrokerEndpointOpt).get.sizeInBytes def writeTo(buffer: ByteBuffer) { buffer.putInt(correlationId) buffer.putShort(errorCode) - coordinatorOpt.orElse(ConsumerMetadataResponse.NoBrokerOpt).foreach(_.writeTo(buffer)) + coordinatorOpt.orElse(ConsumerMetadataResponse.NoBrokerEndpointOpt).foreach(_.writeTo(buffer)) } def describe(details: Boolean) = toString diff --git a/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala b/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala index 5be393ab8272a..33c107f9fea40 100644 --- a/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala +++ b/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala @@ -18,42 +18,47 @@ package kafka.api import java.nio.ByteBuffer + +import kafka.common.{ErrorMapping, TopicAndPartition} import kafka.api.ApiUtils._ -import collection.mutable.ListBuffer -import kafka.network.{BoundedByteBufferSend, RequestChannel} -import kafka.common.{TopicAndPartition, ErrorMapping} +import kafka.network.{RequestOrResponseSend, RequestChannel} import kafka.network.RequestChannel.Response import kafka.utils.Logging object ControlledShutdownRequest extends Logging { - val CurrentVersion = 0.shortValue + val CurrentVersion = 1.shortValue val DefaultClientId = "" def readFrom(buffer: ByteBuffer): ControlledShutdownRequest = { val versionId = buffer.getShort val correlationId = buffer.getInt + val clientId = if (versionId > 0) Some(readShortString(buffer)) else None val brokerId = buffer.getInt - new ControlledShutdownRequest(versionId, correlationId, brokerId) + new ControlledShutdownRequest(versionId, correlationId, clientId, brokerId) } + } -case class ControlledShutdownRequest(val versionId: Short, - val correlationId: Int, - val brokerId: Int) +case class ControlledShutdownRequest(versionId: Short, + correlationId: Int, + clientId: Option[String], + brokerId: Int) extends RequestOrResponse(Some(RequestKeys.ControlledShutdownKey)){ - def this(correlationId: Int, brokerId: Int) = - this(ControlledShutdownRequest.CurrentVersion, correlationId, brokerId) + if (versionId > 0 && clientId.isEmpty) + throw new IllegalArgumentException("`clientId` must be defined if `versionId` > 0") def writeTo(buffer: ByteBuffer) { buffer.putShort(versionId) buffer.putInt(correlationId) + clientId.foreach(writeShortString(buffer, _)) buffer.putInt(brokerId) } - def sizeInBytes(): Int = { - 2 + /* version id */ + def sizeInBytes: Int = { + 2 + /* version id */ 4 + /* correlation id */ + clientId.fold(0)(shortStringLength) 4 /* broker id */ } @@ -63,7 +68,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 = { @@ -71,7 +76,8 @@ case class ControlledShutdownRequest(val versionId: Short, controlledShutdownRequest.append("Name: " + this.getClass.getSimpleName) controlledShutdownRequest.append("; Version: " + versionId) controlledShutdownRequest.append("; CorrelationId: " + correlationId) + controlledShutdownRequest.append(";ClientId:" + clientId) 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 5e0a1cf4f407a..9ecdee73c171b 100644 --- a/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala +++ b/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala @@ -39,9 +39,9 @@ object ControlledShutdownResponse { } -case class ControlledShutdownResponse(val correlationId: Int, - val errorCode: Short = ErrorMapping.NoError, - val partitionsRemaining: Set[TopicAndPartition]) +case class ControlledShutdownResponse(correlationId: Int, + errorCode: Short = ErrorMapping.NoError, + partitionsRemaining: Set[TopicAndPartition]) extends RequestOrResponse() { def sizeInBytes(): Int ={ var size = @@ -68,4 +68,4 @@ case class ControlledShutdownResponse(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 59c09155dd25f..36e288f7a1b15 100644 --- a/core/src/main/scala/kafka/api/FetchRequest.scala +++ b/core/src/main/scala/kafka/api/FetchRequest.scala @@ -30,9 +30,8 @@ import scala.collection.immutable.Map case class PartitionFetchInfo(offset: Long, fetchSize: Int) - object FetchRequest { - val CurrentVersion = 0.shortValue + val CurrentVersion = 1.shortValue val DefaultMaxWait = 0 val DefaultMinBytes = 0 val DefaultCorrelationId = 0 @@ -150,7 +149,7 @@ case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion, (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 = { @@ -171,7 +170,7 @@ case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion, @nonthreadsafe class FetchRequestBuilder() { private val correlationId = new AtomicInteger(0) - private val versionId = FetchRequest.CurrentVersion + private var versionId = FetchRequest.CurrentVersion private var clientId = ConsumerConfig.DefaultClientId private var replicaId = Request.OrdinaryConsumerId private var maxWait = FetchRequest.DefaultMaxWait @@ -206,6 +205,11 @@ class FetchRequestBuilder() { this } + def requestVersion(versionId: Short): FetchRequestBuilder = { + this.versionId = versionId + this + } + def build() = { val fetchRequest = FetchRequest(versionId, correlationId.getAndIncrement, clientId, replicaId, maxWait, minBytes, requestMap.toMap) requestMap.clear() diff --git a/core/src/main/scala/kafka/api/FetchResponse.scala b/core/src/main/scala/kafka/api/FetchResponse.scala index af9308737bf78..aa15612bd5d7f 100644 --- a/core/src/main/scala/kafka/api/FetchResponse.scala +++ b/core/src/main/scala/kafka/api/FetchResponse.scala @@ -22,8 +22,11 @@ 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.{SSLTransportLayer, TransportLayer, Send, MultiSend} + +import scala.collection._ object FetchResponsePartitionData { def readFrom(buffer: ByteBuffer): FetchResponsePartitionData = { @@ -50,9 +53,10 @@ case class FetchResponsePartitionData(error: Short = ErrorMapping.NoError, hw: L class PartitionDataSend(val partitionId: Int, val partitionData: FetchResponsePartitionData) extends Send { + private val emptyBuffer = ByteBuffer.allocate(0) private val messageSize = partitionData.messages.sizeInBytes private var messagesSentSize = 0 - + private var pending = false private val buffer = ByteBuffer.allocate( 4 /** partitionId **/ + FetchResponsePartitionData.headerSize) buffer.putInt(partitionId) buffer.putShort(partitionData.error) @@ -60,19 +64,30 @@ 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 && !pending + + override def destination: String = "" - override def writeTo(channel: GatheringByteChannel): Int = { - var written = 0 - if(buffer.hasRemaining) + override def writeTo(channel: GatheringByteChannel): Long = { + var written = 0L + if (buffer.hasRemaining) written += channel.write(buffer) - if(!buffer.hasRemaining && messagesSentSize < messageSize) { - val bytesSent = partitionData.messages.writeTo(channel, messagesSentSize, messageSize - messagesSentSize) - messagesSentSize += bytesSent - written += bytesSent + if (!buffer.hasRemaining) { + if (messagesSentSize < messageSize) { + val bytesSent = partitionData.messages.writeTo(channel, messagesSentSize, messageSize - messagesSentSize) + messagesSentSize += bytesSent + written += bytesSent + } + if (messagesSentSize >= messageSize && hasPendingWrites(channel)) + channel.write(emptyBuffer) } + + pending = hasPendingWrites(channel) + written } + + override def size = buffer.capacity() + messageSize } object TopicData { @@ -99,31 +114,44 @@ 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 val emptyBuffer = ByteBuffer.allocate(0) + + private var sent = 0L - private var sent = 0 + private var pending = false - override def complete = sent >= size + override def completed: Boolean = sent >= size && !pending + + 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)))) - def writeTo(channel: GatheringByteChannel): Int = { - expectIncomplete() - var written = 0 - if(buffer.hasRemaining) + override def writeTo(channel: GatheringByteChannel): Long = { + if (completed) + throw new KafkaException("This operation cannot be completed on a complete request.") + + var written = 0L + if (buffer.hasRemaining) written += channel.write(buffer) - if(!buffer.hasRemaining && !sends.complete) { - written += sends.writeTo(channel) + if (!buffer.hasRemaining) { + if (!sends.completed) + written += sends.writeTo(channel) + if (sends.completed && hasPendingWrites(channel)) + written += channel.write(emptyBuffer) } + + pending = hasPendingWrites(channel) + sent += written written } @@ -132,12 +160,10 @@ class TopicDataSend(val topicData: TopicData) extends Send { object FetchResponse { - val headerSize = - 4 + /* correlationId */ - 4 /* topic count */ - - def readFrom(buffer: ByteBuffer): FetchResponse = { + // The request version is used to determine which fields we can expect in the response + def readFrom(buffer: ByteBuffer, requestVersion: Int): FetchResponse = { val correlationId = buffer.getInt + val throttleTime = if (requestVersion > 0) buffer.getInt else 0 val topicCount = buffer.getInt val pairs = (1 to topicCount).flatMap(_ => { val topicData = TopicData.readFrom(buffer) @@ -146,27 +172,64 @@ object FetchResponse { (TopicAndPartition(topicData.topic, partitionId), partitionData) } }) - FetchResponse(correlationId, Map(pairs:_*)) + FetchResponse(correlationId, Map(pairs:_*), requestVersion, throttleTime) } -} + // Returns the size of the response header + def headerSize(requestVersion: Int): Int = { + val throttleTimeSize = if (requestVersion > 0) 4 else 0 + 4 + /* correlationId */ + 4 + /* topic count */ + throttleTimeSize + } + + // Returns the size of entire fetch response in bytes (including the header size) + def responseSize(dataGroupedByTopic: Map[String, Map[TopicAndPartition, FetchResponsePartitionData]], + requestVersion: Int): Int = { + headerSize(requestVersion) + + dataGroupedByTopic.foldLeft(0) { case (folded, (topic, partitionDataMap)) => + val topicData = TopicData(topic, partitionDataMap.map { + case (topicAndPartition, partitionData) => (topicAndPartition.partition, partitionData) + }) + folded + topicData.sizeInBytes + } + } +} case class FetchResponse(correlationId: Int, - data: Map[TopicAndPartition, FetchResponsePartitionData]) { + data: Map[TopicAndPartition, FetchResponsePartitionData], + requestVersion: Int = 0, + throttleTimeMs: Int = 0) + extends RequestOrResponse() { /** * Partitions the data into a map of maps (one for each topic). */ - lazy val dataGroupedByTopic = data.groupBy(_._1.topic) + lazy val dataGroupedByTopic = data.groupBy{ case (topicAndPartition, fetchData) => topicAndPartition.topic } + val headerSizeInBytes = FetchResponse.headerSize(requestVersion) + lazy val sizeInBytes = FetchResponse.responseSize(dataGroupedByTopic, requestVersion) - val sizeInBytes = - FetchResponse.headerSize + - dataGroupedByTopic.foldLeft(0) ((folded, curr) => { - val topicData = TopicData(curr._1, curr._2.map { - case (topicAndPartition, partitionData) => (topicAndPartition.partition, partitionData) - }) - folded + topicData.sizeInBytes - }) + /* + * Writes the header of the FetchResponse to the input buffer + */ + def writeHeaderTo(buffer: ByteBuffer) = { + buffer.putInt(sizeInBytes) + buffer.putInt(correlationId) + // Include the throttleTime only if the client can read it + if (requestVersion > 0) + buffer.putInt(throttleTimeMs) + + buffer.putInt(dataGroupedByTopic.size) // topic count + } + /* + * 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) @@ -189,38 +252,50 @@ 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 emptyBuffer = ByteBuffer.allocate(0) + + private val payloadSize = fetchResponse.sizeInBytes - private var sent = 0 + private var sent = 0L - private val sendSize = 4 /* for size */ + size + private var pending = false - override def complete = sent >= sendSize + override def size = 4 /* for size byte */ + payloadSize - private val buffer = ByteBuffer.allocate(4 /* for size */ + FetchResponse.headerSize) - buffer.putInt(size) - buffer.putInt(fetchResponse.correlationId) - buffer.putInt(fetchResponse.dataGroupedByTopic.size) // topic count + override def completed = sent >= size && !pending + + override def destination = dest + + // The throttleTimeSize will be 0 if the request was made from a client sending a V0 style request + private val buffer = ByteBuffer.allocate(4 /* for size */ + fetchResponse.headerSizeInBytes) + fetchResponse.writeHeaderTo(buffer) 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 - } + })) - def writeTo(channel: GatheringByteChannel):Int = { - expectIncomplete() - var written = 0 - if(buffer.hasRemaining) + override def writeTo(channel: GatheringByteChannel): Long = { + if (completed) + throw new KafkaException("This operation cannot be completed on a complete request.") + + var written = 0L + + if (buffer.hasRemaining) written += channel.write(buffer) - if(!buffer.hasRemaining && !sends.complete) { - written += sends.writeTo(channel) + if (!buffer.hasRemaining) { + if (!sends.completed) + written += sends.writeTo(channel) + if (sends.completed && hasPendingWrites(channel)) + written += channel.write(emptyBuffer) } + sent += written + pending = hasPendingWrites(channel) + written } } - diff --git a/core/src/main/scala/kafka/api/GenericRequestAndHeader.scala b/core/src/main/scala/kafka/api/GenericRequestAndHeader.scala index f40e19f4b2ad7..b0c6d7a339850 100644 --- a/core/src/main/scala/kafka/api/GenericRequestAndHeader.scala +++ b/core/src/main/scala/kafka/api/GenericRequestAndHeader.scala @@ -36,7 +36,7 @@ private[kafka] abstract class GenericRequestAndHeader(val versionId: Short, 2 /* version id */ + 4 /* correlation id */ + (2 + clientId.length) /* client id */ + - body.sizeOf(); + body.sizeOf() } override def toString(): String = { @@ -52,4 +52,4 @@ private[kafka] abstract class GenericRequestAndHeader(val versionId: Short, strBuffer.append("; Body: " + body.toString) strBuffer.toString() } -} \ No newline at end of file +} diff --git a/core/src/main/scala/kafka/api/GenericResponseAndHeader.scala b/core/src/main/scala/kafka/api/GenericResponseAndHeader.scala index a4879e26b5362..748b5e935235f 100644 --- a/core/src/main/scala/kafka/api/GenericResponseAndHeader.scala +++ b/core/src/main/scala/kafka/api/GenericResponseAndHeader.scala @@ -29,7 +29,7 @@ private[kafka] abstract class GenericResponseAndHeader(val correlationId: Int, def sizeInBytes(): Int = { 4 /* correlation id */ + - body.sizeOf(); + body.sizeOf() } override def toString(): String = { @@ -43,4 +43,4 @@ private[kafka] abstract class GenericResponseAndHeader(val correlationId: Int, strBuffer.append("; Body: " + body.toString) strBuffer.toString() } -} \ No newline at end of file +} diff --git a/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala b/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala deleted file mode 100644 index f168d9fc99ce5..0000000000000 --- a/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala +++ /dev/null @@ -1,45 +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.api - -import java.nio.ByteBuffer -import kafka.network.{BoundedByteBufferSend, RequestChannel} -import kafka.common.ErrorMapping -import org.apache.kafka.common.requests.{HeartbeatResponse, HeartbeatRequest} -import kafka.api.ApiUtils._ -import kafka.network.RequestChannel.Response -import scala.Some - -object HeartbeatRequestAndHeader { - def readFrom(buffer: ByteBuffer): HeartbeatRequestAndHeader = { - val versionId = buffer.getShort - val correlationId = buffer.getInt - val clientId = readShortString(buffer) - val body = HeartbeatRequest.parse(buffer) - new HeartbeatRequestAndHeader(versionId, correlationId, clientId, body) - } -} - -case class HeartbeatRequestAndHeader(override val versionId: Short, - override val correlationId: Int, - override val clientId: String, - override val body: HeartbeatRequest) - extends GenericRequestAndHeader(versionId, correlationId, clientId, body, RequestKeys.nameForKey(RequestKeys.HeartbeatKey), Some(RequestKeys.HeartbeatKey)) { - - override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { - val errorResponseBody = new HeartbeatResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) - val errorHeartBeatResponseAndHeader = new HeartbeatResponseAndHeader(correlationId, errorResponseBody) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorHeartBeatResponseAndHeader))) - } -} diff --git a/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala b/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala deleted file mode 100644 index 3651e8603dd0e..0000000000000 --- a/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala +++ /dev/null @@ -1,45 +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.api - -import java.nio.ByteBuffer -import kafka.network.{BoundedByteBufferSend, RequestChannel} -import kafka.common.ErrorMapping -import org.apache.kafka.common.requests._ -import kafka.api.ApiUtils._ -import kafka.network.RequestChannel.Response -import scala.Some - -object JoinGroupRequestAndHeader { - def readFrom(buffer: ByteBuffer): JoinGroupRequestAndHeader = { - val versionId = buffer.getShort - val correlationId = buffer.getInt - val clientId = readShortString(buffer) - val body = JoinGroupRequest.parse(buffer) - new JoinGroupRequestAndHeader(versionId, correlationId, clientId, body) - } -} - -case class JoinGroupRequestAndHeader(override val versionId: Short, - override val correlationId: Int, - override val clientId: String, - override val body: JoinGroupRequest) - extends GenericRequestAndHeader(versionId, correlationId, clientId, body, RequestKeys.nameForKey(RequestKeys.JoinGroupKey), Some(RequestKeys.JoinGroupKey)) { - - override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { - val errorResponseBody = new JoinGroupResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) - val errorHeartBeatResponseAndHeader = new JoinGroupResponseAndHeader(correlationId, errorResponseBody) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorHeartBeatResponseAndHeader))) - } -} diff --git a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala index 4ff7e8f8cc695..c2584e0c94301 100644 --- a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala +++ b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala @@ -19,14 +19,16 @@ 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 { @@ -59,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) { @@ -120,9 +122,9 @@ 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) } @@ -134,10 +136,10 @@ case class LeaderAndIsrRequest (versionId: Short, controllerId: Int, controllerEpoch: Int, partitionStateInfos: Map[(String, Int), PartitionStateInfo], - leaders: Set[Broker]) + 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) @@ -184,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 = { @@ -200,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/OffsetCommitRequest.scala b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala index 861a6cf11dc6b..5b362ef7a7652 100644 --- a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala @@ -18,24 +18,24 @@ package kafka.api import java.nio.ByteBuffer + import kafka.api.ApiUtils._ -import kafka.utils.{SystemTime, Logging} -import kafka.network.{RequestChannel, BoundedByteBufferSend} -import kafka.common.{OffsetAndMetadata, ErrorMapping, TopicAndPartition} +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 = 1 + val CurrentVersion: Short = 2 val DefaultClientId = "" def readFrom(buffer: ByteBuffer): OffsetCommitRequest = { - val now = SystemTime.milliseconds - // Read values from the envelope val versionId = buffer.getShort - assert(versionId == 0 || versionId == 1, - "Version " + versionId + " is invalid for OffsetCommitRequest. Valid versions are 0 or 1.") + 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) @@ -43,13 +43,25 @@ object OffsetCommitRequest extends Logging { // Read the OffsetRequest val consumerGroupId = readShortString(buffer) - // version 1 specific fields - var groupGenerationId: Int = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_GENERATION_ID - var consumerId: String = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_CONSUMER_ID - if (versionId == 1) { - groupGenerationId = buffer.getInt - consumerId = 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(_ => { @@ -59,14 +71,19 @@ object OffsetCommitRequest extends Logging { val partitionId = buffer.getInt val offset = buffer.getLong val timestamp = { - val given = buffer.getLong - if (given == -1L) now else given + // 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), OffsetAndMetadata(offset, metadata, timestamp)) }) }) - OffsetCommitRequest(consumerGroupId, immutable.Map(pairs:_*), versionId, correlationId, clientId, groupGenerationId, consumerId) + + OffsetCommitRequest(consumerGroupId, immutable.Map(pairs:_*), versionId, correlationId, clientId, groupGenerationId, consumerId, retentionMs) } } @@ -76,29 +93,14 @@ case class OffsetCommitRequest(groupId: String, 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) + 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, - "Version " + versionId + " is invalid for OffsetCommitRequest. Valid versions are 0 or 1.") - lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic) + assert(versionId == 0 || versionId == 1 || versionId == 2, + "Version " + versionId + " is invalid for OffsetCommitRequest. Valid versions are 0, 1 or 2.") - def filterLargeMetadata(maxMetadataSize: Int) = - requestInfo.filter(info => info._2.metadata == null || info._2.metadata.length <= maxMetadataSize) - - def responseFor(errorCode: Short, offsetMetadataMaxSize: Int) = { - val commitStatus = requestInfo.map {info => - (info._1, if (info._2.metadata != null && info._2.metadata.length > offsetMetadataMaxSize) - ErrorMapping.OffsetMetadataTooLargeCode - else if (errorCode == ErrorMapping.UnknownTopicOrPartitionCode) - ErrorMapping.ConsumerCoordinatorNotAvailableCode - else if (errorCode == ErrorMapping.NotLeaderForPartitionCode) - ErrorMapping.NotCoordinatorForConsumerCode - else - errorCode) - }.toMap - OffsetCommitResponse(commitStatus, correlationId) - } + lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic) def writeTo(buffer: ByteBuffer) { // Write envelope @@ -109,11 +111,17 @@ case class OffsetCommitRequest(groupId: String, // Write OffsetCommitRequest writeShortString(buffer, groupId) // consumer group - // version 1 specific data - if (versionId == 1) { + // 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 @@ -121,7 +129,9 @@ case class OffsetCommitRequest(groupId: String, t1._2.foreach( t2 => { buffer.putInt(t2._1.partition) buffer.putLong(t2._2.offset) - buffer.putLong(t2._2.timestamp) + // version 1 specific data + if (versionId == 1) + buffer.putLong(t2._2.commitTimestamp) writeShortString(buffer, t2._2.metadata) }) }) @@ -132,7 +142,8 @@ case class OffsetCommitRequest(groupId: String, 4 + /* correlationId */ shortStringLength(clientId) + shortStringLength(groupId) + - (if (versionId == 1) 4 /* group generation id */ + shortStringLength(consumerId) else 0) + + (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 @@ -143,15 +154,17 @@ case class OffsetCommitRequest(groupId: String, innerCount + 4 /* partition */ + 8 /* offset */ + - 8 /* timestamp */ + + (if (versionId == 1) 8 else 0) /* timestamp */ + shortStringLength(offsetAndMetadata._2.metadata) }) }) override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { val errorCode = ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) - val errorResponse = responseFor(errorCode, Int.MaxValue) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + 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 = { @@ -163,6 +176,7 @@ case class OffsetCommitRequest(groupId: String, 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() diff --git a/core/src/main/scala/kafka/api/OffsetCommitResponse.scala b/core/src/main/scala/kafka/api/OffsetCommitResponse.scala index 624a1c1cc5406..116547ae1139e 100644 --- a/core/src/main/scala/kafka/api/OffsetCommitResponse.scala +++ b/core/src/main/scala/kafka/api/OffsetCommitResponse.scala @@ -20,7 +20,7 @@ package kafka.api import java.nio.ByteBuffer import kafka.utils.Logging -import kafka.common.TopicAndPartition +import kafka.common.{ErrorMapping, TopicAndPartition} object OffsetCommitResponse extends Logging { val CurrentVersion: Short = 0 @@ -47,6 +47,8 @@ case class OffsetCommitResponse(commitStatus: Map[TopicAndPartition, Short], 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(commitStatusGroupedByTopic.size) diff --git a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala index c7604b9cdeb8f..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.{OffsetAndMetadata, 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 = { @@ -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 = OffsetAndMetadata.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 = { diff --git a/core/src/main/scala/kafka/api/OffsetRequest.scala b/core/src/main/scala/kafka/api/OffsetRequest.scala index 3d483bc7518ad..d2c1c95c5713b 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 @@ -114,10 +115,10 @@ case class OffsetRequest(requestInfo: Map[TopicAndPartition, PartitionOffsetRequ override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { val partitionOffsetResponseMap = requestInfo.map { case (topicAndPartition, partitionOffsetRequest) => - (topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), null)) + (topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), Nil)) } 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/ProducerRequest.scala b/core/src/main/scala/kafka/api/ProducerRequest.scala index b2366e7eedcac..7fb143e489913 100644 --- a/core/src/main/scala/kafka/api/ProducerRequest.scala +++ b/core/src/main/scala/kafka/api/ProducerRequest.scala @@ -18,14 +18,15 @@ 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 + val CurrentVersion = 1.shortValue def readFrom(buffer: ByteBuffer): ProducerRequest = { val versionId: Short = buffer.getShort @@ -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 a286272c834b6..7719f3054004e 100644 --- a/core/src/main/scala/kafka/api/ProducerResponse.scala +++ b/core/src/main/scala/kafka/api/ProducerResponse.scala @@ -23,6 +23,7 @@ import kafka.common.{TopicAndPartition, ErrorMapping} import kafka.api.ApiUtils._ object ProducerResponse { + // readFrom assumes that the response is written using V1 format def readFrom(buffer: ByteBuffer): ProducerResponse = { val correlationId = buffer.getInt val topicCount = buffer.getInt @@ -37,14 +38,17 @@ object ProducerResponse { }) }) - ProducerResponse(correlationId, Map(statusPairs:_*)) + val throttleTime = buffer.getInt + ProducerResponse(correlationId, Map(statusPairs:_*), ProducerRequest.CurrentVersion, throttleTime) } } case class ProducerResponseStatus(var error: Short, offset: Long) case class ProducerResponse(correlationId: Int, - status: Map[TopicAndPartition, ProducerResponseStatus]) + status: Map[TopicAndPartition, ProducerResponseStatus], + requestVersion: Int = 0, + throttleTime: Int = 0) extends RequestOrResponse() { /** @@ -55,6 +59,7 @@ case class ProducerResponse(correlationId: Int, def hasError = status.values.exists(_.error != ErrorMapping.NoError) val sizeInBytes = { + val throttleTimeSize = if (requestVersion > 0) 4 else 0 val groupedStatus = statusGroupedByTopic 4 + /* correlation id */ 4 + /* topic count */ @@ -67,7 +72,8 @@ case class ProducerResponse(correlationId: Int, 2 + /* error code */ 8 /* offset */ } - }) + }) + + throttleTimeSize } def writeTo(buffer: ByteBuffer) { @@ -86,6 +92,9 @@ case class ProducerResponse(correlationId: Int, buffer.putLong(nextOffset) } }) + // Throttle time is only supported on V1 style requests + if (requestVersion > 0) + buffer.putInt(throttleTime) } override def describe(details: Boolean):String = { toString } diff --git a/core/src/main/scala/kafka/api/RequestKeys.scala b/core/src/main/scala/kafka/api/RequestKeys.scala index c24c0345feedc..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 @@ -46,9 +48,7 @@ object RequestKeys { ControlledShutdownKey -> ("ControlledShutdown", ControlledShutdownRequest.readFrom), OffsetCommitKey -> ("OffsetCommit", OffsetCommitRequest.readFrom), OffsetFetchKey -> ("OffsetFetch", OffsetFetchRequest.readFrom), - ConsumerMetadataKey -> ("ConsumerMetadata", ConsumerMetadataRequest.readFrom), - JoinGroupKey -> ("JoinGroup", JoinGroupRequestAndHeader.readFrom), - HeartbeatKey -> ("Heartbeat", HeartbeatRequestAndHeader.readFrom) + ConsumerMetadataKey -> ("ConsumerMetadata", ConsumerMetadataRequest.readFrom) ) def nameForKey(key: Short): String = { @@ -61,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/StopReplicaRequest.scala b/core/src/main/scala/kafka/api/StopReplicaRequest.scala index 5e14987c990fe..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 @@ -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 3431f3f65d1ae..2fc3c9585fbc6 100644 --- a/core/src/main/scala/kafka/api/StopReplicaResponse.scala +++ b/core/src/main/scala/kafka/api/StopReplicaResponse.scala @@ -42,9 +42,9 @@ object StopReplicaResponse { } -case class StopReplicaResponse(val correlationId: Int, - val responseMap: Map[TopicAndPartition, Short], - val errorCode: Short = ErrorMapping.NoError) +case class StopReplicaResponse(correlationId: Int, + responseMap: Map[TopicAndPartition, Short], + errorCode: Short = ErrorMapping.NoError) extends RequestOrResponse() { def sizeInBytes(): Int ={ var size = @@ -72,4 +72,4 @@ case class StopReplicaResponse(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 0190076df0adf..7b56b31d697a2 100644 --- a/core/src/main/scala/kafka/api/TopicMetadata.scala +++ b/core/src/main/scala/kafka/api/TopicMetadata.scala @@ -17,25 +17,24 @@ package kafka.api -import kafka.cluster.Broker +import kafka.cluster.BrokerEndPoint import java.nio.ByteBuffer import kafka.api.ApiUtils._ import kafka.utils.Logging import kafka.common._ -import org.apache.kafka.common.utils.Utils._ 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: Array[PartitionMetadata] = new Array[PartitionMetadata](numPartitions) for(i <- 0 until numPartitions) { val partitionMetadata = PartitionMetadata.readFrom(buffer, brokers) - partitionsMetadata(partitionMetadata.partitionId) = partitionMetadata + partitionsMetadata(i) = partitionMetadata } new TopicMetadata(topic, partitionsMetadata, errorCode) } @@ -89,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 @@ -110,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 */ + @@ -142,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 + " (" + formatAddress(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 7dca09ce637a4..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,10 +48,10 @@ object TopicMetadataRequest extends Logging { } } -case class TopicMetadataRequest(val versionId: Short, - val correlationId: Int, - val clientId: String, - val topics: Seq[String]) +case class TopicMetadataRequest(versionId: Short, + correlationId: Int, + clientId: String, + topics: Seq[String]) extends RequestOrResponse(Some(RequestKeys.MetadataKey)){ def this(topics: Seq[String], correlationId: Int) = @@ -80,7 +82,7 @@ case class TopicMetadataRequest(val versionId: Short, topic => TopicMetadata(topic, Nil, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) } val errorResponse = TopicMetadataResponse(Seq(), topicMetadata, 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 = { @@ -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 92ac4e687be22..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,7 +25,7 @@ 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)) @@ -33,7 +33,7 @@ object TopicMetadataResponse { } } -case class TopicMetadataResponse(brokers: Seq[Broker], +case class TopicMetadataResponse(brokers: Seq[BrokerEndPoint], topicsMetadata: Seq[TopicMetadata], correlationId: Int) extends RequestOrResponse() { diff --git a/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala b/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala index 530982e36b179..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,7 +51,13 @@ 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) } @@ -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/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala old mode 100644 new mode 100755 index ebba87f056668..68c7e7f594f2f --- a/core/src/main/scala/kafka/client/ClientUtils.scala +++ b/core/src/main/scala/kafka/client/ClientUtils.scala @@ -14,21 +14,22 @@ * 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.{ErrorMapping, KafkaException} -import kafka.utils.{Utils, Logging} +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 -import org.apache.kafka.common.utils.Utils.{getHost, getPort} +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) @@ -42,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) @@ -73,7 +74,7 @@ object ClientUtils extends Logging{ } else { debug("Successfully fetched metadata for %d topic(s) %s".format(topics.size, topics)) } - return topicMetadataResponse + topicMetadataResponse } /** @@ -83,7 +84,7 @@ 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(_.connectionString).mkString(",")) @@ -96,11 +97,11 @@ 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) + def parseBrokerList(brokerListStr: String): Seq[BrokerEndPoint] = { + val brokersStr = CoreUtils.parseCsvList(brokerListStr) brokersStr.zipWithIndex.map { case (address, brokerId) => - new Broker(brokerId, getHost(address), getPort(address)) + BrokerEndPoint.createBrokerEndPoint(brokerId, address) } } @@ -111,7 +112,7 @@ object ClientUtils extends Logging{ var channel: BlockingChannel = null var connected = false while (!connected) { - val allBrokers = getAllBrokersInCluster(zkClient) + val allBrokers = getAllBrokerEndPointsForChannel(zkClient, SecurityProtocol.PLAINTEXT) Random.shuffle(allBrokers).find { broker => trace("Connecting to broker %s:%d.".format(broker.host, broker.port)) try { @@ -143,7 +144,7 @@ object ClientUtils extends Logging{ while (!offsetManagerChannelOpt.isDefined) { - var coordinatorOpt: Option[Broker] = None + var coordinatorOpt: Option[BrokerEndPoint] = None while (!coordinatorOpt.isDefined) { try { @@ -152,7 +153,7 @@ object ClientUtils extends Logging{ 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.buffer) + val consumerMetadataResponse = ConsumerMetadataResponse.readFrom(response.payload()) debug("Consumer metadata response: " + consumerMetadataResponse.toString) if (consumerMetadataResponse.errorCode == ErrorMapping.NoError) coordinatorOpt = consumerMetadataResponse.coordinatorOpt 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 0060add008bb3..79e16c167f67c --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -17,18 +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 org.apache.kafka.common.utils.Utils._ + +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. */ 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)) @@ -36,9 +62,21 @@ 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)) } @@ -47,36 +85,60 @@ 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) } } -case class Broker(id: Int, host: String, port: Int) { - - override def toString: 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 connectionString: String = formatAddress(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..75efd77770121 --- /dev/null +++ b/core/src/main/scala/kafka/cluster/BrokerEndPoint.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.cluster + +import java.nio.ByteBuffer + +import kafka.api.ApiUtils._ +import kafka.common.KafkaException +import org.apache.kafka.common.utils.Utils._ + +object BrokerEndPoint { + + private val uriParseExp = """\[?([0-9a-z\-.:]*)\]?:([0-9]+)""".r + + /** + * BrokerEndPoint URI is host:port or [ipv6_host]:port + * Note that unlike EndPoint (or listener) this URI has no security information. + */ + def parseHostPort(connectionString: String): Option[(String, Int)] = { + connectionString match { + case uriParseExp(host, port) => try Some(host, port.toInt) catch { case e: NumberFormatException => None } + case _ => None + } + } + + /** + * BrokerEndPoint URI is host:port or [ipv6_host]:port + * Note that unlike EndPoint (or listener) this URI has no security information. + */ + def createBrokerEndPoint(brokerId: Int, connectionString: String): BrokerEndPoint = { + parseHostPort(connectionString).map { case (host, port) => new BrokerEndPoint(brokerId, host, port) }.getOrElse { + 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 ff106b47e6ee1..ee332ed4ddccb --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -17,24 +17,22 @@ package kafka.cluster import kafka.common._ -import kafka.admin.AdminUtils import kafka.utils._ +import kafka.utils.CoreUtils.{inReadLock,inWriteLock} +import kafka.admin.AdminUtils import kafka.api.{PartitionStateInfo, LeaderAndIsr} import kafka.log.LogConfig -import kafka.server.{TopicPartitionRequestKey, LogOffsetMetadata, OffsetManager, ReplicaManager} +import kafka.server._ import kafka.metrics.KafkaMetricsGroup import kafka.controller.KafkaController import kafka.message.ByteBufferMessageSet import java.io.IOException import java.util.concurrent.locks.ReentrantReadWriteLock -import kafka.utils.Utils.{inReadLock,inWriteLock} -import scala.Some 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 */ @@ -52,6 +50,7 @@ class Partition(val topic: String, @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. @@ -61,14 +60,15 @@ class Partition(val topic: String, this.logIdent = "Partition [%s,%d] on broker %d: ".format(topic, partitionId, localBrokerId) 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 = { @@ -86,12 +86,13 @@ 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.fetchEntityConfig(zkClient, ConfigType.Topic, topic)) val log = logManager.createLog(TopicAndPartition(topic, partitionId), config) 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) @@ -142,6 +143,7 @@ class Partition(val topic: String, 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) @@ -159,8 +161,7 @@ class Partition(val topic: String, * and setting the new leader and ISR */ def makeLeader(controllerId: Int, - partitionStateInfo: PartitionStateInfo, correlationId: Int, - offsetManager: OffsetManager): Boolean = { + partitionStateInfo: PartitionStateInfo, correlationId: Int): Boolean = { inWriteLock(leaderIsrUpdateLock) { val allReplicas = partitionStateInfo.allReplicas val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch @@ -181,11 +182,10 @@ class Partition(val topic: String, val newLeaderReplica = getReplica().get newLeaderReplica.convertHWToLocalOffsetMetadata() // reset log end offset for remote replicas - assignedReplicas.foreach(r => if (r.brokerId != localBrokerId) r.logEndOffset = LogOffsetMetadata.UnknownOffsetMetadata) + 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(newLeaderReplica) - if (topic == OffsetManager.OffsetsTopicName) - offsetManager.loadOffsetsFromLog(partitionId) true } } @@ -196,7 +196,7 @@ class Partition(val topic: String, */ def makeFollower(controllerId: Int, partitionStateInfo: PartitionStateInfo, - correlationId: Int, offsetManager: OffsetManager): Boolean = { + correlationId: Int): Boolean = { inWriteLock(leaderIsrUpdateLock) { val allReplicas = partitionStateInfo.allReplicas val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch @@ -213,13 +213,6 @@ class Partition(val topic: String, leaderEpoch = leaderAndIsr.leaderEpoch zkVersion = leaderAndIsr.zkVersion - leaderReplicaIdOpt.foreach { leaderReplica => - if (topic == OffsetManager.OffsetsTopicName && - /* if we are making a leader->follower transition */ - leaderReplica == localBrokerId) - offsetManager.clearOffsetsInPartition(partitionId) - } - if (leaderReplicaIdOpt.isDefined && leaderReplicaIdOpt.get == newLeaderBrokerId) { false } @@ -230,54 +223,102 @@ class Partition(val topic: String, } } - def updateLeaderHWAndMaybeExpandIsr(replicaId: Int) { + /** + * 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 high watermark - if (!inSyncReplicas.contains(replica) && - assignedReplicas.map(_.brokerId).contains(replicaId) && - replica.logEndOffset.offsetDiff(leaderHW) >= 0) { - // 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) = { + /* + * 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) - r.logEndOffset.messageOffset >= requiredOffset + if (r.logEndOffset.messageOffset >= requiredOffset) { + trace("Replica %d of %s-%d received offset %d".format(r.brokerId, topic, partitionId, requiredOffset)) + true + } + else + false else true /* also count the local (leader) replica */ }) - trace("%d/%d acks satisfied for %s-%d".format(numAcks, requiredAcks, topic, partitionId)) - if ((requiredAcks < 0 && leaderReplica.highWatermark.messageOffset >= requiredOffset) || - (requiredAcks > 0 && numAcks >= requiredAcks)) { + + 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 ) { /* - * requiredAcks < 0 means acknowledge after all replicas in ISR - * are fully caught up to the (local) leader's offset - * corresponding to this produce request. + * 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 */ - (true, ErrorMapping.NoError) + if (minIsr <= curInSyncReplicas.size) { + (true, ErrorMapping.NoError) + } else { + (true, ErrorMapping.NotEnoughReplicasAfterAppendCode) + } } else (false, ErrorMapping.NoError) case None => @@ -286,8 +327,14 @@ class Partition(val topic: String, } /** - * 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) @@ -296,21 +343,21 @@ class Partition(val topic: String, if(oldHighWatermark.precedes(newHighWatermark)) { leaderReplica.highWatermark = newHighWatermark debug("High watermark for partition [%s,%d] updated to %s".format(topic, partitionId, newHighWatermark)) - // some delayed requests may be unblocked after HW changed - val requestKey = new TopicPartitionRequestKey(this.topic, this.partitionId) - replicaManager.unblockDelayedFetchRequests(requestKey) - replicaManager.unblockDelayedProduceRequests(requestKey) + // 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(","))) } } - def maybeShrinkIsr(replicaMaxLagTimeMs: Long, replicaMaxLagMessages: Long) { + 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) @@ -327,38 +374,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.messageOffset >= 0 && - leaderLogEndOffset.messageOffset - r.logEndOffset.messageOffset > 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) = { + 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.unblockDelayedFetchRequests(new TopicPartitionRequestKey(this.topic, this.partitionId)) + 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 @@ -373,7 +428,9 @@ class Partition(val topic: String, val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, newIsr.map(r => r.brokerId).toList, zkVersion) val (updateSucceeded,newVersion) = ReplicationUtils.updateLeaderAndIsr(zkClient, topic, partitionId, newLeaderAndIsr, controllerEpoch, zkVersion) + if(updateSucceeded) { + replicaManager.recordIsrChange(new TopicAndPartition(topic, partitionId)) inSyncReplicas = newIsr zkVersion = newVersion trace("ISR updated to [%s] and zkVersion updated to [%d]".format(newIsr.mkString(","), zkVersion)) @@ -382,6 +439,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 diff --git a/core/src/main/scala/kafka/cluster/Replica.scala b/core/src/main/scala/kafka/cluster/Replica.scala index bd13c20338ce3..740e83567c1a1 100644 --- a/core/src/main/scala/kafka/cluster/Replica.scala +++ b/core/src/main/scala/kafka/cluster/Replica.scala @@ -19,7 +19,7 @@ package kafka.cluster import kafka.log.Log import kafka.utils.{SystemTime, Time, Logging} -import kafka.server.LogOffsetMetadata +import kafka.server.{LogReadResult, LogOffsetMetadata} import kafka.common.KafkaException import java.util.concurrent.atomic.AtomicLong @@ -34,8 +34,6 @@ class Replica(val brokerId: Int, // 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 - // the time when log offset is updated - private[this] val logEndOffsetUpdateTimeMsValue = new AtomicLong(time.milliseconds) val topic = partition.topic val partitionId = partition.partitionId @@ -47,12 +45,27 @@ class Replica(val brokerId: Int, } } - def logEndOffset_=(newLogEndOffset: LogOffsetMetadata) { + 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) + } + } + + private def logEndOffset_=(newLogEndOffset: LogOffsetMetadata) { if (isLocal) { throw new KafkaException("Should not set log end offset on partition [%s,%d]'s local replica %d".format(topic, partitionId, brokerId)) } else { logEndOffsetMetadata = newLogEndOffset - logEndOffsetUpdateTimeMsValue.set(time.milliseconds) trace("Setting log end offset for replica %d for partition [%s,%d] to [%s]" .format(brokerId, topic, partitionId, logEndOffsetMetadata)) } @@ -64,8 +77,6 @@ class Replica(val brokerId: Int, else logEndOffsetMetadata - def logEndOffsetUpdateTimeMs = logEndOffsetUpdateTimeMsValue.get() - def highWatermark_=(newHighWatermark: LogOffsetMetadata) { if (isLocal) { highWatermarkMetadata = newHighWatermark 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..8e2f49d1fd0e7 --- /dev/null +++ b/core/src/main/scala/kafka/common/AppInfo.scala @@ -0,0 +1,54 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 com.yammer.metrics.core.Gauge +import kafka.metrics.KafkaMetricsGroup +import org.apache.kafka.common.utils.AppInfoParser + +object AppInfo extends KafkaMetricsGroup { + private var isRegistered = false + private val lock = new Object() + + def registerInfo(): Unit = { + lock.synchronized { + if (isRegistered) { + return + } + } + + newGauge("Version", + new Gauge[String] { + def value = { + AppInfoParser.getVersion() + } + }) + + newGauge("CommitID", + new Gauge[String] { + def value = { + AppInfoParser.getCommitId(); + } + }) + + lock.synchronized { + isRegistered = true + } + + } +} diff --git a/core/src/main/scala/kafka/common/AuthorizationException.scala b/core/src/main/scala/kafka/common/AuthorizationException.scala new file mode 100644 index 0000000000000..009cf1a21061e --- /dev/null +++ b/core/src/main/scala/kafka/common/AuthorizationException.scala @@ -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 kafka.common + +/** + * Exception thrown when a principal is not authorized to perform an operation. + * @param message + */ +class AuthorizationException(message: String) extends RuntimeException(message) { +} diff --git a/core/src/main/scala/kafka/common/BaseEnum.scala b/core/src/main/scala/kafka/common/BaseEnum.scala new file mode 100644 index 0000000000000..9c39466b7f378 --- /dev/null +++ b/core/src/main/scala/kafka/common/BaseEnum.scala @@ -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. + */ +package kafka.common + +/* + * We inherit from `Product` and `Serializable` because `case` objects and classes inherit from them and if we don't + * do it here, the compiler will infer types that unexpectedly include `Product` and `Serializable`, see + * http://underscore.io/blog/posts/2015/06/04/more-on-sealed.html for more information. + */ +trait BaseEnum extends Product with Serializable { + def name: String +} 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/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 3fae7910e4ce1..23224eca9370b 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 @@ -17,13 +17,14 @@ package kafka.common -import kafka.message.InvalidMessageException import java.nio.ByteBuffer -import java.lang.Throwable + +import kafka.message.InvalidMessageException + 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) @@ -47,8 +48,20 @@ object ErrorMapping { 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 + // 23: INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY + // 24: UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY + // 25: UNKNOWN_CONSUMER_ID + // 26: INVALID_SESSION_TIMEOUT + // 27: COMMITTING_PARTITIONS_NOT_ASSIGNED + // 28: INVALID_COMMIT_OFFSET_SIZE + val AuthorizationCode: Short = 29; - private val exceptionToCode = + private val exceptionToCode = Map[Class[Throwable], Short]( classOf[OffsetOutOfRangeException].asInstanceOf[Class[Throwable]] -> OffsetOutOfRangeCode, classOf[InvalidMessageException].asInstanceOf[Class[Throwable]] -> InvalidMessageCode, @@ -65,18 +78,24 @@ object ErrorMapping { 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[InvalidTopicException].asInstanceOf[Class[Throwable]] -> InvalidTopicCode, + classOf[MessageSetSizeTooLargeException].asInstanceOf[Class[Throwable]] -> MessageSetSizeTooLargeCode, + classOf[NotEnoughReplicasException].asInstanceOf[Class[Throwable]] -> NotEnoughReplicasCode, + classOf[NotEnoughReplicasAfterAppendException].asInstanceOf[Class[Throwable]] -> NotEnoughReplicasAfterAppendCode, + classOf[AuthorizationException].asInstanceOf[Class[Throwable]] -> AuthorizationCode ).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/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 1586243d20d6a..deb48b1cee5e2 100644 --- a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala +++ b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala @@ -17,39 +17,64 @@ package kafka.common -case class OffsetAndMetadata(offset: Long, - metadata: String = OffsetAndMetadata.NoMetadata, - timestamp: Long = -1L) { - override def toString = "OffsetAndMetadata[%d,%s%s]" - .format(offset, - if (metadata != null && metadata.length > 0) metadata else "NO_METADATA", - if (timestamp == -1) "" else "," + timestamp.toString) +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 OffsetAndMetadata { +object OffsetMetadata { val InvalidOffset: Long = -1L val NoMetadata: String = "" - val InvalidTime: Long = -1L + + val InvalidOffsetMetadata = OffsetMetadata(OffsetMetadata.InvalidOffset, OffsetMetadata.NoMetadata) } -case class OffsetMetadataAndError(offset: Long, - metadata: String = OffsetAndMetadata.NoMetadata, - error: Short = ErrorMapping.NoError) { +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 this(offsetMetadata: OffsetAndMetadata, error: Short) = - this(offsetMetadata.offset, offsetMetadata.metadata, error) + def offset = offsetMetadata.offset - def this(error: Short) = - this(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, error) + def metadata = offsetMetadata.metadata - def asTuple = (offset, metadata, error) + override def toString = "[%s,CommitTime %d,ExpirationTime %d]".format(offsetMetadata, commitTimestamp, expireTimestamp) +} - override def toString = "OffsetMetadataAndError[%d,%s,%d]".format(offset, metadata, error) +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 + + def metadata = offsetMetadata.metadata + + override def toString = "[%s,ErrorCode %d]".format(offsetMetadata, error) } object OffsetMetadataAndError { - val NoOffset = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.NoError) - val OffsetsLoading = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.OffsetsLoadInProgressCode) - val NotOffsetManagerForGroup = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.NotCoordinatorForConsumerCode) + 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/common/Topic.scala b/core/src/main/scala/kafka/common/Topic.scala index ad759786d1c22..db75d4b37a394 100644 --- a/core/src/main/scala/kafka/common/Topic.scala +++ b/core/src/main/scala/kafka/common/Topic.scala @@ -18,7 +18,7 @@ package kafka.common import util.matching.Regex -import kafka.server.OffsetManager +import kafka.coordinator.ConsumerCoordinator object Topic { @@ -26,7 +26,7 @@ object Topic { private val maxNameLength = 255 private val rgx = new Regex(legalChars + "+") - val InternalTopics = Set(OffsetManager.OffsetsTopicName) + val InternalTopics = Set(ConsumerCoordinator.OffsetsTopicName) def validate(topic: String) { if (topic.length <= 0) @@ -43,4 +43,26 @@ object Topic { case None => throw new InvalidTopicException("topic name " + topic + " is illegal, contains a character other than ASCII alphanumerics, '.', '_' and '-'") } } + + /** + * Due to limitations in metric names, topics with a period ('.') or underscore ('_') could collide. + * + * @param topic The topic to check for colliding character + * @return true if the topic has collision characters + */ + def hasCollisionChars(topic: String): Boolean = { + topic.contains("_") || topic.contains(".") + } + + /** + * Returns true if the topicNames collide due to a period ('.') or underscore ('_') in the same position. + * + * @param topicA A topic to check for collision + * @param topicB A topic to check for collision + * @return true if the topics collide + */ + def hasCollision(topicA: String, topicB: String): Boolean = { + topicA.replace('.', '_') == topicB.replace('.', '_') + } + } 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..95db1dcb5d7af 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 @@ -33,5 +34,4 @@ case class TopicAndPartition(topic: String, partition: Int) { def asTuple = (topic, partition) override def toString = "[%s,%d]".format(topic, 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/common/ZkNodeChangeNotificationListener.scala b/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala new file mode 100644 index 0000000000000..eb44c31d83447 --- /dev/null +++ b/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.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.common + +import kafka.utils.{Time, SystemTime, ZkUtils, Logging} +import org.I0Itec.zkclient.{IZkChildListener, ZkClient} +import scala.collection.JavaConverters._ + +/** + * Handle the notificationMessage. + */ +trait NotificationHandler { + def processNotification(notificationMessage: String) +} + +/** + * A listener that subscribes to seqNodeRoot for any child changes where all children are assumed to be sequence node + * with seqNodePrefix. When a child is added under seqNodeRoot this class gets notified, it looks at lastExecutedChange + * number to avoid duplicate processing and if it finds an unprocessed child, it reads its data and calls supplied + * notificationHandler's processNotification() method with the child's data as argument. As part of processing these changes it also + * purges any children with currentTime - createTime > changeExpirationMs. + * + * The caller/user of this class should ensure that they use zkClient.subscribeStateChanges and call processAllNotifications + * method of this class from ZkStateChangeListener's handleNewSession() method. This is necessary to ensure that if zk session + * is terminated and reestablished any missed notification will be processed immediately. + * @param zkClient + * @param seqNodeRoot + * @param seqNodePrefix + * @param notificationHandler + * @param changeExpirationMs + * @param time + */ +class ZkNodeChangeNotificationListener(private val zkClient: ZkClient, + private val seqNodeRoot: String, + private val seqNodePrefix: String, + private val notificationHandler: NotificationHandler, + private val changeExpirationMs: Long = 15 * 60 * 1000, + private val time: Time = SystemTime) extends Logging { + private var lastExecutedChange = -1L + + /** + * create seqNodeRoot and begin watching for any new children nodes. + */ + def init() { + ZkUtils.makeSurePersistentPathExists(zkClient, seqNodeRoot) + zkClient.subscribeChildChanges(seqNodeRoot, NodeChangeListener) + processAllNotifications() + } + + /** + * Process all changes + */ + def processAllNotifications() { + val changes = zkClient.getChildren(seqNodeRoot) + processNotifications(changes.asScala.sorted) + } + + /** + * Process the given list of notifications + */ + private def processNotifications(notifications: Seq[String]) { + if (notifications.nonEmpty) { + info(s"Processing notification(s) to $seqNodeRoot") + val now = time.milliseconds + for (notification <- notifications) { + val changeId = changeNumber(notification) + if (changeId > lastExecutedChange) { + val changeZnode = seqNodeRoot + "/" + notification + val (data, stat) = ZkUtils.readDataMaybeNull(zkClient, changeZnode) + data map (notificationHandler.processNotification(_)) getOrElse(logger.warn(s"read null data from $changeZnode when processing notification $notification")) + } + lastExecutedChange = changeId + } + purgeObsoleteNotifications(now, notifications) + } + } + + /** + * Purges expired notifications. + * @param now + * @param notifications + */ + private def purgeObsoleteNotifications(now: Long, notifications: Seq[String]) { + for (notification <- notifications.sorted) { + val notificationNode = seqNodeRoot + "/" + notification + val (data, stat) = ZkUtils.readDataMaybeNull(zkClient, notificationNode) + if (data.isDefined) { + if (now - stat.getCtime > changeExpirationMs) { + debug(s"Purging change notification $notificationNode") + ZkUtils.deletePath(zkClient, notificationNode) + } + } + } + } + + /* get the change number from a change notification znode */ + private def changeNumber(name: String): Long = name.substring(seqNodePrefix.length).toLong + + /** + * A listener that gets invoked when a node is created to notify changes. + */ + object NodeChangeListener extends IZkChildListener { + override def handleChildChange(path: String, notifications: java.util.List[String]) { + try { + import scala.collection.JavaConverters._ + if (notifications != null) + processNotifications(notifications.asScala.sorted) + } catch { + case e: Exception => error(s"Error processing notification change for path = $path and notification= $notifications :", e) + } + } + } + +} + diff --git a/core/src/main/scala/kafka/consumer/BaseConsumer.scala b/core/src/main/scala/kafka/consumer/BaseConsumer.scala new file mode 100644 index 0000000000000..4e956bbd15b23 --- /dev/null +++ b/core/src/main/scala/kafka/consumer/BaseConsumer.scala @@ -0,0 +1,82 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 + +/** + * A base consumer used to abstract both old and new consumer + * this class should be removed (along with BaseProducer) be removed + * once we deprecate old consumer + */ +trait BaseConsumer { + def receive(): BaseConsumerRecord + def stop() + def cleanup() +} + +case class BaseConsumerRecord(topic: String, partition: Int, offset: Long, key: Array[Byte], value: Array[Byte]) + +class NewShinyConsumer(topic: String, consumerProps: Properties) extends BaseConsumer { + import org.apache.kafka.clients.consumer.KafkaConsumer + import scala.collection.JavaConversions._ + + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](consumerProps) + consumer.subscribe(List(topic)) + var recordIter = consumer.poll(0).iterator + + override def receive(): BaseConsumerRecord = { + while (!recordIter.hasNext) + recordIter = consumer.poll(Long.MaxValue).iterator + + val record = recordIter.next + BaseConsumerRecord(record.topic, record.partition, record.offset, record.key, record.value) + } + + override def stop() { + this.consumer.wakeup() + } + + override def cleanup() { + this.consumer.close() + } +} + +class OldConsumer(topicFilter: TopicFilter, consumerProps: Properties) extends BaseConsumer { + import kafka.serializer.DefaultDecoder + + val consumerConnector = Consumer.create(new ConsumerConfig(consumerProps)) + val stream: KafkaStream[Array[Byte], Array[Byte]] = + consumerConnector.createMessageStreamsByFilter(topicFilter, 1, new DefaultDecoder(), new DefaultDecoder()).head + val iter = stream.iterator + + override def receive(): BaseConsumerRecord = { + // we do not need to check hasNext for KafkaStream iterator + val messageAndMetadata = iter.next + BaseConsumerRecord(messageAndMetadata.topic, messageAndMetadata.partition, messageAndMetadata.offset, messageAndMetadata.key, messageAndMetadata.message) + } + + override def stop() { + this.consumerConnector.shutdown() + } + + override def cleanup() { + this.consumerConnector.shutdown() + } +} + diff --git a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala index 9ebbee6c16dc8..97a56ce7f2acb 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala @@ -58,6 +58,7 @@ object ConsumerConfig extends Config { validateGroupId(config.groupId) validateAutoOffsetReset(config.autoOffsetReset) validateOffsetsStorage(config.offsetsStorage) + validatePartitionAssignmentStrategy(config.partitionAssignmentStrategy) } def validateClientId(clientId: String) { @@ -85,6 +86,15 @@ object ConsumerConfig extends 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) { @@ -104,8 +114,6 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig( /** the socket timeout for network requests. Its value should be at least fetch.wait.max.ms. */ val socketTimeoutMs = props.getInt("socket.timeout.ms", SocketTimeout) - require(fetchWaitMaxMs <= socketTimeoutMs, "socket.timeout.ms should always be at least fetch.wait.max.ms" + - " to prevent unnecessary socket timeouts") /** the socket receive buffer for network requests */ val socketReceiveBufferBytes = props.getInt("socket.receive.buffer.bytes", SocketBufferSize) @@ -133,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) diff --git a/core/src/main/scala/kafka/consumer/ConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ConsumerConnector.scala index 07677c1c26768..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 = true) + 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..8801ff8ee825c 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala @@ -17,45 +17,69 @@ package kafka.consumer -import kafka.cluster.Broker -import kafka.server.AbstractFetcherThread +import kafka.api.{OffsetRequest, Request, FetchRequestBuilder, FetchResponsePartitionData} +import kafka.cluster.BrokerEndPoint import kafka.message.ByteBufferMessageSet -import kafka.api.{Request, OffsetRequest, FetchResponsePartitionData} -import kafka.common.TopicAndPartition - +import kafka.server.{PartitionFetchState, AbstractFetcherThread} +import kafka.common.{ErrorMapping, TopicAndPartition} +import scala.collection.JavaConverters +import JavaConverters._ +import ConsumerFetcherThread._ 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, - fetchSize = config.fetchMessageMaxBytes, - fetcherBrokerId = Request.OrdinaryConsumerId, - maxWait = config.fetchWaitMaxMs, - minBytes = config.fetchMinBytes, + fetchBackOffMs = config.refreshLeaderBackoffMs, isInterruptible = true) { + type REQ = FetchRequest + type PD = PartitionData + + private val clientId = config.clientId + private val fetchSize = config.fetchMessageMaxBytes + + private val simpleConsumer = new SimpleConsumer(sourceBroker.host, sourceBroker.port, config.socketTimeoutMs, + config.socketReceiveBufferBytes, config.clientId) + + private val fetchRequestBuilder = new FetchRequestBuilder(). + clientId(clientId). + replicaId(Request.OrdinaryConsumerId). + maxWait(config.fetchWaitMaxMs). + minBytes(config.fetchMinBytes). + requestVersion(kafka.api.FetchRequest.CurrentVersion) + + override def initiateShutdown(): Boolean = { + val justShutdown = super.initiateShutdown() + if (justShutdown && isInterruptible) + simpleConsumer.disconnectToHandleJavaIOBug() + justShutdown + } + + override def shutdown(): Unit = { + super.shutdown() + simpleConsumer.close() + } + // process fetched data - def processPartitionData(topicAndPartition: TopicAndPartition, fetchOffset: Long, partitionData: FetchResponsePartitionData) { + def processPartitionData(topicAndPartition: TopicAndPartition, fetchOffset: Long, partitionData: PartitionData) { val pti = partitionMap(topicAndPartition) if (pti.getFetchOffset != fetchOffset) throw new RuntimeException("Offset doesn't match for partition [%s,%d] pti offset: %d fetch offset: %d" .format(topicAndPartition.topic, topicAndPartition.partition, pti.getFetchOffset, fetchOffset)) - pti.enqueue(partitionData.messages.asInstanceOf[ByteBufferMessageSet]) + pti.enqueue(partitionData.underlying.messages.asInstanceOf[ByteBufferMessageSet]) } // handle a partition whose offset is out of range and return a new fetch offset def handleOffsetOutOfRange(topicAndPartition: TopicAndPartition): Long = { - var startTimestamp : Long = 0 - config.autoOffsetReset match { - case OffsetRequest.SmallestTimeString => startTimestamp = OffsetRequest.EarliestTime - case OffsetRequest.LargestTimeString => startTimestamp = OffsetRequest.LatestTime - case _ => startTimestamp = OffsetRequest.LatestTime + val startTimestamp = config.autoOffsetReset match { + case OffsetRequest.SmallestTimeString => OffsetRequest.EarliestTime + case OffsetRequest.LargestTimeString => OffsetRequest.LatestTime + case _ => OffsetRequest.LatestTime } val newOffset = simpleConsumer.earliestOrLatestOffset(topicAndPartition, startTimestamp, Request.OrdinaryConsumerId) val pti = partitionMap(topicAndPartition) @@ -69,4 +93,37 @@ class ConsumerFetcherThread(name: String, removePartitions(partitions.toSet) consumerFetcherManager.addPartitionsWithError(partitions) } + + protected def buildFetchRequest(partitionMap: collection.Map[TopicAndPartition, PartitionFetchState]): FetchRequest = { + partitionMap.foreach { case ((topicAndPartition, partitionFetchState)) => + if (partitionFetchState.isActive) + fetchRequestBuilder.addFetch(topicAndPartition.topic, topicAndPartition.partition, partitionFetchState.offset, + fetchSize) + } + + new FetchRequest(fetchRequestBuilder.build()) + } + + protected def fetch(fetchRequest: FetchRequest): collection.Map[TopicAndPartition, PartitionData] = + simpleConsumer.fetch(fetchRequest.underlying).data.map { case (key, value) => + key -> new PartitionData(value) + } + +} + +object ConsumerFetcherThread { + + class FetchRequest(val underlying: kafka.api.FetchRequest) extends AbstractFetcherThread.FetchRequest { + def isEmpty: Boolean = underlying.requestInfo.isEmpty + def offset(topicAndPartition: TopicAndPartition): Long = underlying.requestInfo(topicAndPartition).offset + } + + class PartitionData(val underlying: FetchResponsePartitionData) extends AbstractFetcherThread.PartitionData { + def errorCode: Short = underlying.error + def toByteBufferMessageSet: ByteBufferMessageSet = underlying.messages.asInstanceOf[ByteBufferMessageSet] + def highWatermark: Long = underlying.hw + def exception: Option[Throwable] = + if (errorCode == ErrorMapping.NoError) None else Some(ErrorMapping.exceptionFor(errorCode)) + + } } 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 f63e6c59bb1e8..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)) } } diff --git a/core/src/main/scala/kafka/consumer/FetchRequestAndResponseStats.scala b/core/src/main/scala/kafka/consumer/FetchRequestAndResponseStats.scala index 5243f415288c6..05ea9ac3de716 100644 --- a/core/src/main/scala/kafka/consumer/FetchRequestAndResponseStats.scala +++ b/core/src/main/scala/kafka/consumer/FetchRequestAndResponseStats.scala @@ -19,13 +19,22 @@ package kafka.consumer import java.util.concurrent.TimeUnit -import kafka.common.ClientIdAndBroker +import kafka.common.{ClientIdAllBrokers, ClientIdBroker, ClientIdAndBroker} import kafka.metrics.{KafkaMetricsGroup, KafkaTimer} import kafka.utils.Pool -class FetchRequestAndResponseMetrics(metricId: ClientIdAndBroker) extends KafkaMetricsGroup { - val requestTimer = new KafkaTimer(newTimer(metricId + "FetchRequestRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS)) - val requestSizeHist = newHistogram(metricId + "FetchResponseSize") +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) + val throttleTimeStats = newTimer("FetchRequestThrottleRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS, tags) } /** @@ -33,14 +42,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)) } } @@ -56,7 +65,7 @@ object FetchRequestAndResponseStatsRegistry { } def removeConsumerFetchRequestAndResponseStats(clientId: String) { - val pattern = (clientId + "-ConsumerFetcherThread.*").r + val pattern = (".*" + clientId + ".*").r val keys = globalStats.keys for (key <- keys) { pattern.findFirstIn(key) match { diff --git a/core/src/main/scala/kafka/consumer/PartitionAssignor.scala b/core/src/main/scala/kafka/consumer/PartitionAssignor.scala old mode 100644 new mode 100755 index 8ea7368dc394a..849284ad2cfa0 --- a/core/src/main/scala/kafka/consumer/PartitionAssignor.scala +++ b/core/src/main/scala/kafka/consumer/PartitionAssignor.scala @@ -19,16 +19,18 @@ package kafka.consumer import org.I0Itec.zkclient.ZkClient import kafka.common.TopicAndPartition -import kafka.utils.{Utils, ZkUtils, Logging} +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 consumer thread. This only includes assignments for threads that belong - * to the given assignment-context's consumer. + * @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): scala.collection.Map[TopicAndPartition, ConsumerThreadId] + def assign(ctx: AssignmentContext): Pool[String, mutable.Map[TopicAndPartition, ConsumerThreadId]] } @@ -69,43 +71,51 @@ class AssignmentContext(group: String, val consumerId: String, excludeInternalTo class RoundRobinAssignor() extends PartitionAssignor with Logging { def assign(ctx: AssignmentContext) = { - val partitionOwnershipDecision = collection.mutable.Map[TopicAndPartition, ConsumerThreadId]() - - // 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 = Utils.circularIterator(headThreadIdSet.toSeq.sorted) + 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)) + } - 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) + 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 }) - }.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() - if (threadId.consumer == ctx.consumerId) - partitionOwnershipDecision += (topicPartition -> threadId) - }) - - partitionOwnershipDecision + + 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 } } @@ -121,9 +131,10 @@ class RoundRobinAssignor() extends PartitionAssignor with Logging { class RangeAssignor() extends PartitionAssignor with Logging { def assign(ctx: AssignmentContext) = { - val partitionOwnershipDecision = collection.mutable.Map[TopicAndPartition, ConsumerThreadId]() - - for ((topic, consumerThreadIdSet) <- ctx.myTopicThreadIds) { + 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) @@ -133,7 +144,7 @@ class RangeAssignor() extends PartitionAssignor with Logging { info("Consumer " + ctx.consumerId + " rebalancing the following partitions: " + curPartitions + " for topic " + topic + " with consumers: " + curConsumers) - for (consumerThreadId <- consumerThreadIdSet) { + for (consumerThreadId <- curConsumers) { val myConsumerPosition = curConsumers.indexOf(consumerThreadId) assert(myConsumerPosition >= 0) val startPart = nPartsPerConsumer * myConsumerPosition + myConsumerPosition.min(nConsumersWithExtraPart) @@ -150,12 +161,15 @@ class RangeAssignor() extends PartitionAssignor with Logging { val partition = curPartitions(i) info(consumerThreadId + " attempting to claim partition " + partition) // record the partition ownership decision - partitionOwnershipDecision += (TopicAndPartition(topic, partition) -> consumerThreadId) + val assignmentForConsumer = partitionAssignment.getAndMaybePut(consumerThreadId.consumer) + assignmentForConsumer += (TopicAndPartition(topic, partition) -> consumerThreadId) } } } } - partitionOwnershipDecision + // 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 d349a3000feb9..b7af6d6c71344 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 java.util.concurrent.TimeUnit + 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._ /** @@ -36,7 +41,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 @@ -56,6 +60,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() @@ -63,14 +77,19 @@ class SimpleConsumer(val host: String, } } - private def sendRequest(request: RequestOrResponse): Receive = { + private def sendRequest(request: RequestOrResponse): NetworkReceive = { lock synchronized { - 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 => info("Reconnect due to socket error: %s".format(e.toString)) // retry once @@ -90,12 +109,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.buffer) + ConsumerMetadataResponse.readFrom(response.payload()) } /** @@ -105,18 +124,20 @@ 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(), request.versionId) val fetchedSize = fetchResponse.sizeInBytes - fetchRequestAndResponseStats.getFetchRequestAndResponseStats(brokerInfo).requestSizeHist.update(fetchedSize) + fetchRequestAndResponseStats.getFetchRequestAndResponseStats(host, port).requestSizeHist.update(fetchedSize) fetchRequestAndResponseStats.getFetchRequestAndResponseAllBrokersStats.requestSizeHist.update(fetchedSize) + fetchRequestAndResponseStats.getFetchRequestAndResponseStats(host, port).throttleTimeStats.update(fetchResponse.throttleTimeMs, TimeUnit.MILLISECONDS) + fetchRequestAndResponseStats.getFetchRequestAndResponseAllBrokersStats.throttleTimeStats.update(fetchResponse.throttleTimeMs, TimeUnit.MILLISECONDS) fetchResponse } @@ -125,25 +146,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) = { // 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).buffer) + 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 0954b3c3ff8b3..6994c8e89055b --- a/core/src/main/scala/kafka/consumer/TopicCount.scala +++ b/core/src/main/scala/kafka/consumer/TopicCount.scala @@ -19,7 +19,7 @@ package kafka.consumer import scala.collection._ import org.I0Itec.zkclient.ZkClient -import kafka.utils.{Json, ZKGroupDirs, ZkUtils, Logging, Utils} +import kafka.utils.{Json, ZKGroupDirs, ZkUtils, Logging, CoreUtils} import kafka.common.KafkaException private[kafka] trait TopicCount { @@ -136,7 +136,7 @@ private[kafka] class WildcardTopicCount(zkClient: ZkClient, TopicCount.makeConsumerThreadIdsPerTopic(consumerIdString, Map(wildcardTopics.map((_, numStreams)): _*)) } - def getTopicCountMap = Map(Utils.JSONEscapeString(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/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala old mode 100644 new mode 100755 index fbc680fde21b0..2027ec8aadadf --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -28,25 +28,26 @@ 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.Utils.inLock +import kafka.utils.CoreUtils.inLock import kafka.utils.ZkUtils._ import kafka.utils._ import org.I0Itec.zkclient.exception.ZkNodeExistsException -import org.I0Itec.zkclient.{IZkChildListener, IZkDataListener, IZkStateListener, ZkClient} +import org.I0Itec.zkclient.{IZkChildListener, IZkDataListener, IZkStateListener, ZkClient, ZkConnection} import org.apache.zookeeper.Watcher.Event.KeeperState import scala.collection._ - +import scala.collection.JavaConversions._ /** * This class handles the consumers interaction with zookeeper * * 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. @@ -88,6 +89,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private val rebalanceLock = new Object private var fetcher: Option[ConsumerFetcherManager] = None private var zkClient: ZkClient = null + private var zkConnection : ZkConnection = null private var topicRegistry = new Pool[String, Pool[Int, PartitionTopicInfo]] private val checkpointedZkOffsets = new Pool[TopicAndPartition, Long] private val topicThreadIdAndQueues = new Pool[(String, ConsumerThreadId), BlockingQueue[FetchedDataChunk]] @@ -102,11 +104,12 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, 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(config.clientId + "-KafkaCommitsPerSec", "commits", TimeUnit.SECONDS) - private val zkCommitMeter = newMeter(config.clientId + "-ZooKeeperCommitsPerSec", "commits", TimeUnit.SECONDS) - private val rebalanceTimer = new KafkaTimer(newTimer(config.clientId + "-RebalanceRateAndTime", TimeUnit.MILLISECONDS, TimeUnit.SECONDS)) + 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 @@ -138,6 +141,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } KafkaMetricsReporter.startReporters(config.props) + AppInfo.registerInfo() def this(config: ConsumerConfig) = this(config, true) @@ -160,6 +164,13 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, 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)) @@ -167,14 +178,17 @@ 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) + val (client, connection) = ZkUtils.createZkClientAndConnection(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs) + zkClient = client + zkConnection = connection } // 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) + offsetsChannel = ClientUtils.channelToOffsetManager(config.groupId, zkClient, + config.offsetsChannelSocketTimeoutMs, config.offsetsChannelBackoffMs) debug("Connected to offset manager %s:%d.".format(offsetsChannel.host, offsetsChannel.port)) } @@ -198,7 +212,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } sendShutdownToAllQueues() if (config.autoCommitEnable) - commitOffsets() + commitOffsets(true) if (zkClient != null) { zkClient.close() zkClient = null @@ -249,14 +263,17 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, val timestamp = SystemTime.milliseconds.toString val consumerRegistrationInfo = Json.encode(Map("version" -> 1, "subscription" -> topicCount.getTopicCountMap, "pattern" -> topicCount.pattern, "timestamp" -> timestamp)) + val zkWatchedEphemeral = new ZKCheckedEphemeral(dirs. + consumerRegistryDir + "/" + consumerIdString, + consumerRegistrationInfo, + zkConnection.getZookeeper) + zkWatchedEphemeral.create() - createEphemeralPathExpectConflictHandleZKBug(zkClient, dirs.consumerRegistryDir + "/" + consumerIdString, consumerRegistrationInfo, null, - (consumerZKString, consumer) => true, config.zkSessionTimeoutMs) info("end registering consumer " + consumerIdString + " in ZK") } private def sendShutdownToAllQueues() = { - for (queue <- topicThreadIdAndQueues.values) { + for (queue <- topicThreadIdAndQueues.values.toSet[BlockingQueue[FetchedDataChunk]]) { debug("Clearing up queue") queue.clear() queue.put(ZookeeperConsumerConnector.shutdownCommand) @@ -285,21 +302,34 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } } - def commitOffsets(isAutoCommit: Boolean = true) { - var retriesRemaining = 1 + (if (isAutoCommit) config.offsetsCommitMaxRetries else 0) // no retries for commits from auto-commit - var done = false + /** + * KAFKA-1743: This method added for backward compatibility. + */ + def commitOffsets { commitOffsets(true) } - while (!done) { - val committed = offsetsChannelLock synchronized { // committed when we receive either no error codes or only MetadataTooLarge errors - val offsetsToCommit = immutable.Map(topicRegistry.flatMap { case (topic, partitionTopicInfos) => - partitionTopicInfos.map { case (partition, info) => - TopicAndPartition(info.topic, info.partitionId) -> OffsetAndMetadata(info.getConsumeOffset()) - } - }.toSeq:_*) + 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) => + offsetsToCommit.foreach { case (topicAndPartition, offsetAndMetadata) => commitOffsetToZooKeeper(topicAndPartition, offsetAndMetadata.offset) } true @@ -309,29 +339,29 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, try { kafkaCommitMeter.mark(offsetsToCommit.size) offsetsChannel.send(offsetCommitRequest) - val offsetCommitResponse = OffsetCommitResponse.readFrom(offsetsChannel.receive().buffer) + 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)) => + 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) + val offset = offsetsToCommit(topicPartition).offset + commitOffsetToZooKeeper(topicPartition, offset) } (folded._1 || // update commitFailed - errorCode != ErrorMapping.NoError, + errorCode != ErrorMapping.NoError, - folded._2 || // update retryableIfFailed - (only metadata too large is not retryable) - (errorCode != ErrorMapping.NoError && errorCode != ErrorMapping.OffsetMetadataTooLargeCode), + 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, + folded._3 || // update shouldRefreshCoordinator + errorCode == ErrorMapping.NotCoordinatorForConsumerCode || + errorCode == ErrorMapping.ConsumerCoordinatorNotAvailableCode, - // update error count - folded._4 + (if (errorCode != ErrorMapping.NoError) 1 else 0)) + // update error count + folded._4 + (if (errorCode != ErrorMapping.NoError) 1 else 0)) } } debug(errorCount + " errors in offset commit response.") @@ -360,11 +390,10 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } } - done = if (isShuttingDown.get() && isAutoCommit) { // should not retry indefinitely if shutting down + done = { retriesRemaining -= 1 retriesRemaining == 0 || committed - } else - true + } if (!done) { debug("Retrying offset commit in %d ms".format(config.offsetsChannelBackoffMs)) @@ -377,7 +406,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, 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, OffsetAndMetadata.NoMetadata, ErrorMapping.NoError)) + case Some(offsetStr) => (topicPartition, OffsetMetadataAndError(offsetStr.toLong)) case None => (topicPartition, OffsetMetadataAndError.NoOffset) } } @@ -397,7 +426,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, ensureOffsetManagerConnected() try { offsetsChannel.send(offsetFetchRequest) - val offsetFetchResponse = OffsetFetchResponse.readFrom(offsetsChannel.receive().buffer) + val offsetFetchResponse = OffsetFetchResponse.readFrom(offsetsChannel.receive().payload()) trace("Offset fetch response: %s.".format(offsetFetchResponse)) val (leaderChanged, loadInProgress) = @@ -483,6 +512,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) @@ -516,14 +548,15 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private var isWatcherTriggered = false private val lock = new ReentrantLock private val cond = lock.newCondition() - + @volatile private var allTopicsOwnedPartitionsCount = 0 - newGauge(config.clientId + "-" + config.groupId + "-AllTopicsOwnedPartitionsCount", new Gauge[Int] { - def value() = allTopicsOwnedPartitionsCount - }) + newGauge("OwnedPartitionsCount", + new Gauge[Int] { + def value() = allTopicsOwnedPartitionsCount + }, + Map("clientId" -> config.clientId, "groupId" -> config.groupId)) - private def ownedPartitionsCountMetricName(topic: String) = - "%s-%s-%s-OwnedPartitionsCount".format(config.clientId, config.groupId, topic) + 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() { @@ -576,7 +609,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, for(partition <- infos.keys) { deletePartitionOwnershipFromZK(topic, partition) } - removeMetric(ownedPartitionsCountMetricName(topic)) + removeMetric("OwnedPartitionsCount", ownedPartitionsCountMetricTags(topic)) localTopicRegistry.remove(topic) } allTopicsOwnedPartitionsCount = 0 @@ -589,36 +622,35 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, def syncedRebalance() { rebalanceLock synchronized { rebalanceTimer.time { - if(isShuttingDown.get()) { - return - } else { - for (i <- 0 until config.rebalanceMaxRetries) { - info("begin rebalancing consumer " + consumerIdString + " try #" + i) - var done = false - var cluster: Cluster = null - try { - cluster = getCluster(zkClient) - done = rebalance(cluster) - } catch { - case e: 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. - **/ - info("exception during rebalance ", e) - } - info("end rebalancing consumer " + consumerIdString + " try #" + i) - if (done) { - return - } else { - /* Here the cache is at a risk of being stale. To take future rebalancing decisions correctly, we should - * clear the cache */ - info("Rebalancing attempt failed. Clearing the cache before the next rebalancing operation is triggered") - } - // stop all fetchers and clear all the queues to avoid data duplication - closeFetchersForQueues(cluster, kafkaMessageAndMetadataStreams, topicThreadIdAndQueues.map(q => q._2)) - Thread.sleep(config.rebalanceBackoffMs) + for (i <- 0 until config.rebalanceMaxRetries) { + if(isShuttingDown.get()) { + return } + info("begin rebalancing consumer " + consumerIdString + " try #" + i) + var done = false + var cluster: Cluster = null + try { + cluster = getCluster(zkClient) + done = rebalance(cluster) + } 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. + **/ + info("exception during rebalance ", e) + } + info("end rebalancing consumer " + consumerIdString + " try #" + i) + if (done) { + return + } else { + /* Here the cache is at a risk of being stale. To take future rebalancing decisions correctly, we should + * clear the cache */ + info("Rebalancing attempt failed. Clearing the cache before the next rebalancing operation is triggered") + } + // stop all fetchers and clear all the queues to avoid data duplication + closeFetchersForQueues(cluster, kafkaMessageAndMetadataStreams, topicThreadIdAndQueues.map(q => q._2)) + Thread.sleep(config.rebalanceBackoffMs) } } } @@ -646,16 +678,26 @@ 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 partitionOwnershipDecision = partitionAssignor.assign(assignmentContext) + 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])) // fetch current offsets for all topic-partitions - val topicPartitions = partitionOwnershipDecision.keySet.toSeq + val topicPartitions = partitionAssignment.keySet.toSeq val offsetFetchResponseOpt = fetchOffsets(topicPartitions) @@ -666,7 +708,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, topicPartitions.foreach(topicAndPartition => { val (topic, partition) = topicAndPartition.asTuple val offset = offsetFetchResponse.requestInfo(topicAndPartition).offset - val threadId = partitionOwnershipDecision(topicAndPartition) + val threadId = partitionAssignment(topicAndPartition) addPartitionTopicInfo(currentTopicRegistry, partition, topic, offset, threadId) }) @@ -674,17 +716,43 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, * 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)) { - allTopicsOwnedPartitionsCount = partitionOwnershipDecision.size + if(reflectPartitionOwnershipDecision(partitionAssignment)) { + allTopicsOwnedPartitionsCount = partitionAssignment.size - partitionOwnershipDecision.view.groupBy { case(topicPartition, consumerThreadId) => topicPartition.topic } + partitionAssignment.view.groupBy { case(topicPartition, consumerThreadId) => topicPartition.topic } .foreach { case (topic, partitionThreadPairs) => - newGauge(ownedPartitionsCountMetricName(topic), new Gauge[Int] { - def value() = partitionThreadPairs.size - }) + newGauge("OwnedPartitionsCount", + new Gauge[Int] { + def value() = partitionThreadPairs.size + }, + ownedPartitionsCountMetricTags(topic)) } topicRegistry = currentTopicRegistry + // Invoke beforeStartingFetchers callback if the consumerRebalanceListener is set. + if (consumerRebalanceListener != null) { + info("Invoking rebalance listener before starting fetchers.") + + // 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 { @@ -702,7 +770,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 @@ -711,8 +778,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 => } } @@ -757,9 +826,9 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } } - private def reflectPartitionOwnershipDecision(partitionOwnershipDecision: Map[TopicAndPartition, ConsumerThreadId]): Boolean = { + private def reflectPartitionOwnershipDecision(partitionAssignment: Map[TopicAndPartition, ConsumerThreadId]): Boolean = { var successfullyOwnedPartitions : List[(String, Int)] = Nil - val partitionOwnershipSuccessful = partitionOwnershipDecision.map { partitionOwner => + val partitionOwnershipSuccessful = partitionAssignment.map { partitionOwner => val topic = partitionOwner._1.topic val partition = partitionOwner._1.partition val consumerThreadId = partitionOwner._2 @@ -863,10 +932,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) ) }) 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 ecbfa0f328ba6..64e33553d2ca0 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -16,22 +16,28 @@ */ package kafka.controller -import kafka.network.{Receive, BlockingChannel} -import kafka.utils.{Utils, Logging, ShutdownableThread} +import kafka.api.{LeaderAndIsr, KAFKA_090, PartitionStateInfo} +import kafka.utils._ +import org.apache.kafka.clients.{ClientResponse, ClientRequest, ManualMetadataUpdater, NetworkClient} +import org.apache.kafka.common.{TopicPartition, Node} +import org.apache.kafka.common.metrics.Metrics +import org.apache.kafka.common.network.{Selectable, ChannelBuilders, Selector, NetworkReceive} +import org.apache.kafka.common.protocol.{SecurityProtocol, ApiKeys} +import org.apache.kafka.common.requests._ +import org.apache.kafka.common.security.ssl.SSLFactory +import org.apache.kafka.common.utils.Time import collection.mutable.HashMap import kafka.cluster.Broker +import java.net.{SocketTimeoutException} 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 kafka.common.{KafkaException, TopicAndPartition} import collection.Set +import collection.JavaConverters._ -class ControllerChannelManager (private val controllerContext: ControllerContext, config: KafkaConfig) extends Logging { - private val brokerStateInfo = new HashMap[Int, ControllerBrokerStateInfo] +class ControllerChannelManager(controllerContext: ControllerContext, config: KafkaConfig, time: Time, metrics: Metrics, threadNamePrefix: Option[String] = None) extends Logging { + protected val brokerStateInfo = new HashMap[Int, ControllerBrokerStateInfo] private val brokerLock = new Object this.logIdent = "[Channel manager on controller " + config.brokerId + "]: " @@ -45,16 +51,16 @@ class ControllerChannelManager (private val controllerContext: ControllerContext def shutdown() = { brokerLock synchronized { - brokerStateInfo.foreach(brokerState => removeExistingBroker(brokerState._1)) + brokerStateInfo.values.foreach(removeExistingBroker) } } - def sendRequest(brokerId : Int, request : RequestOrResponse, callback: (RequestOrResponse) => Unit = null) { + def sendRequest(brokerId: Int, apiKey: ApiKeys, apiVersion: Option[Short], request: AbstractRequest, callback: AbstractRequestResponse => Unit = null) { brokerLock synchronized { val stateInfoOpt = brokerStateInfo.get(brokerId) stateInfoOpt match { case Some(stateInfo) => - stateInfo.messageQueue.put((request, callback)) + stateInfo.messageQueue.put(QueueItem(apiKey, apiVersion, request, callback)) case None => warn("Not sending request %s to broker %d, since it is offline.".format(request, brokerId)) } @@ -73,160 +79,216 @@ class ControllerChannelManager (private val controllerContext: ControllerContext def removeBroker(brokerId: Int) { brokerLock synchronized { - removeExistingBroker(brokerId) + removeExistingBroker(brokerStateInfo(brokerId)) } } private def addNewBroker(broker: Broker) { - val messageQueue = new LinkedBlockingQueue[(RequestOrResponse, (RequestOrResponse) => Unit)](config.controllerMessageQueueSize) - debug("Controller %d trying to connect to broker %d".format(config.brokerId,broker.id)) - val channel = new BlockingChannel(broker.host, broker.port, - BlockingChannel.UseDefaultBufferSize, - BlockingChannel.UseDefaultBufferSize, - config.controllerSocketTimeoutMs) - val requestThread = new RequestSendThread(config.brokerId, controllerContext, broker, messageQueue, channel) + val messageQueue = new LinkedBlockingQueue[QueueItem] + debug("Controller %d trying to connect to broker %d".format(config.brokerId, broker.id)) + val brokerEndPoint = broker.getBrokerEndPoint(config.interBrokerSecurityProtocol) + val brokerNode = new Node(broker.id, brokerEndPoint.host, brokerEndPoint.port) + val networkClient = { + val selector = new Selector( + NetworkReceive.UNLIMITED, + config.connectionsMaxIdleMs, + metrics, + time, + "controller-channel", + Map("broker-id" -> broker.id.toString).asJava, + false, + ChannelBuilders.create(config.interBrokerSecurityProtocol, SSLFactory.Mode.CLIENT, config.channelConfigs) + ) + new NetworkClient( + selector, + new ManualMetadataUpdater(Seq(brokerNode).asJava), + config.brokerId.toString, + 1, + 0, + Selectable.USE_DEFAULT_BUFFER_SIZE, + Selectable.USE_DEFAULT_BUFFER_SIZE, + config.requestTimeoutMs + ) + } + val threadName = threadNamePrefix match { + case None => "Controller-%d-to-broker-%d-send-thread".format(config.brokerId, broker.id) + case Some(name) => "%s:Controller-%d-to-broker-%d-send-thread".format(name,config.brokerId, broker.id) + } + + val requestThread = new RequestSendThread(config.brokerId, controllerContext, broker, messageQueue, networkClient, brokerNode, config, time, threadName) requestThread.setDaemon(false) - brokerStateInfo.put(broker.id, new ControllerBrokerStateInfo(channel, broker, messageQueue, requestThread)) + brokerStateInfo.put(broker.id, new ControllerBrokerStateInfo(networkClient, brokerNode, broker, messageQueue, requestThread)) } - private def removeExistingBroker(brokerId: Int) { + private def removeExistingBroker(brokerState: ControllerBrokerStateInfo) { try { - brokerStateInfo(brokerId).channel.disconnect() - brokerStateInfo(brokerId).messageQueue.clear() - brokerStateInfo(brokerId).requestSendThread.shutdown() - brokerStateInfo.remove(brokerId) - }catch { + brokerState.networkClient.close() + brokerState.messageQueue.clear() + brokerState.requestSendThread.shutdown() + brokerStateInfo.remove(brokerState.broker.id) + } catch { case e: Throwable => error("Error while removing broker by the controller", e) } } - private def startRequestSendThread(brokerId: Int) { + protected def startRequestSendThread(brokerId: Int) { val requestThread = brokerStateInfo(brokerId).requestSendThread if(requestThread.getState == Thread.State.NEW) requestThread.start() } } +case class QueueItem(apiKey: ApiKeys, apiVersion: Option[Short], request: AbstractRequest, callback: AbstractRequestResponse => Unit) + class RequestSendThread(val controllerId: Int, val controllerContext: ControllerContext, val toBroker: Broker, - val queue: BlockingQueue[(RequestOrResponse, (RequestOrResponse) => Unit)], - val channel: BlockingChannel) - extends ShutdownableThread("Controller-%d-to-broker-%d-send-thread".format(controllerId, toBroker.id)) { + val queue: BlockingQueue[QueueItem], + val networkClient: NetworkClient, + val brokerNode: Node, + val config: KafkaConfig, + val time: Time, + name: String) + extends ShutdownableThread(name = name) { + private val lock = new Object() private val stateChangeLogger = KafkaController.stateChangeLogger - connectToBroker(toBroker, channel) + private val socketTimeoutMs = config.controllerSocketTimeoutMs override def doWork(): Unit = { - val queueItem = queue.take() - val request = queueItem._1 - val callback = queueItem._2 - var receive: Receive = null + + def backoff(): Unit = CoreUtils.swallowTrace(Thread.sleep(300)) + + val QueueItem(apiKey, apiVersion, request, callback) = queue.take() + import NetworkClientBlockingOps._ + var clientResponse: ClientResponse = 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) - isSendSuccessful = true + if (!brokerReady()) { + isSendSuccessful = false + backoff() + } + else { + val requestHeader = apiVersion.fold(networkClient.nextRequestHeader(apiKey))(networkClient.nextRequestHeader(apiKey, _)) + val send = new RequestSend(brokerNode.idString, requestHeader, request.toStruct) + val clientRequest = new ClientRequest(time.milliseconds(), true, send, null) + clientResponse = networkClient.blockingSendAndReceive(clientRequest, socketTimeoutMs)(time).getOrElse { + throw new SocketTimeoutException(s"No response received within $socketTimeoutMs ms") + } + isSendSuccessful = true + } } catch { case e: Throwable => // if the send was not successful, reconnect to broker and resend the message - error(("Controller %d epoch %d failed to send request %s to broker %s. " + + warn(("Controller %d epoch %d fails to send request %s to broker %s. " + "Reconnecting to broker.").format(controllerId, controllerContext.epoch, - request.toString, toBroker.toString()), e) - channel.disconnect() - connectToBroker(toBroker, channel) + request.toString, toBroker.toString()), e) + networkClient.close(brokerNode.idString) isSendSuccessful = false - // backoff before retrying the connection and send - Utils.swallow(Thread.sleep(300)) + backoff() } } - 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 %s for a request sent to broker %s" - .format(controllerId, controllerContext.epoch, response.toString, toBroker.toString)) + if (clientResponse != null) { + val response = ApiKeys.forId(clientResponse.request.request.header.apiKey) match { + case ApiKeys.LEADER_AND_ISR => new LeaderAndIsrResponse(clientResponse.responseBody) + case ApiKeys.STOP_REPLICA => new StopReplicaResponse(clientResponse.responseBody) + case ApiKeys.UPDATE_METADATA_KEY => new UpdateMetadataResponse(clientResponse.responseBody) + case apiKey => throw new KafkaException(s"Unexpected apiKey received: $apiKey") + } + 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) - // If there is any socket error (eg, socket timeout), the channel is no longer usable and needs to be recreated. - channel.disconnect() + 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 connection is no longer usable and needs to be recreated. + networkClient.close(brokerNode.idString) } } - private def connectToBroker(broker: Broker, channel: BlockingChannel) { + private def brokerReady(): Boolean = { + import NetworkClientBlockingOps._ try { - channel.connect() - info("Controller %d connected to %s for sending state change requests".format(controllerId, broker.toString())) - } catch { - case e: Throwable => { - channel.disconnect() - error("Controller %d's connection to broker %s was unsuccessful".format(controllerId, broker.toString()), e) + + if (networkClient.isReady(brokerNode, time.milliseconds())) + true + else { + val ready = networkClient.blockingReady(brokerNode, socketTimeoutMs)(time) + + if (!ready) + throw new SocketTimeoutException(s"Failed to connect within $socketTimeoutMs ms") + + info("Controller %d connected to %s for sending state change requests".format(controllerId, toBroker.toString())) + true } + } catch { + case e: Throwable => + error("Controller %d's connection to broker %s was unsuccessful".format(controllerId, toBroker.toString()), e) + networkClient.close(brokerNode.idString) + false } } + } class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging { val controllerContext = controller.controllerContext val controllerId: Int = controller.config.brokerId - val clientId: String = controller.clientId - 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]] + val leaderAndIsrRequestMap = mutable.Map.empty[Int, mutable.Map[TopicPartition, PartitionStateInfo]] + val stopReplicaRequestMap = mutable.Map.empty[Int, Seq[StopReplicaRequestInfo]] + val updateMetadataRequestMap = mutable.Map.empty[Int, mutable.Map[TopicPartition, PartitionStateInfo]] private val stateChangeLogger = KafkaController.stateChangeLogger def newBatch() { // raise error if the previous batch is not empty - if(leaderAndIsrRequestMap.size > 0) + if (leaderAndIsrRequestMap.size > 0) throw new IllegalStateException("Controller to broker state change requests batch is not empty while creating " + "a new one. Some LeaderAndIsr state changes %s might be lost ".format(leaderAndIsrRequestMap.toString())) - if(stopReplicaRequestMap.size > 0) + if (stopReplicaRequestMap.size > 0) throw new IllegalStateException("Controller to broker state change requests batch is not empty while creating a " + "new one. Some StopReplica state changes %s might be lost ".format(stopReplicaRequestMap.toString())) - if(updateMetadataRequestMap.size > 0) + if (updateMetadataRequestMap.size > 0) throw new IllegalStateException("Controller to broker state change requests batch is not empty while creating a " + "new one. Some UpdateMetadata state changes %s might be lost ".format(updateMetadataRequestMap.toString())) } + def clear() { + leaderAndIsrRequestMap.clear() + stopReplicaRequestMap.clear() + updateMetadataRequestMap.clear() + } + def addLeaderAndIsrRequestForBrokers(brokerIds: Seq[Int], topic: String, partition: Int, leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, - replicas: Seq[Int], callback: (RequestOrResponse) => Unit = null) { - 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)) + replicas: Seq[Int], callback: AbstractRequestResponse => Unit = null) { + val topicPartition = new TopicPartition(topic, partition) + + brokerIds.filter(_ >= 0).foreach { brokerId => + val result = leaderAndIsrRequestMap.getOrElseUpdate(brokerId, mutable.Map.empty) + result.put(topicPartition, PartitionStateInfo(leaderIsrAndControllerEpoch, replicas.toSet)) } addUpdateMetadataRequestForBrokers(controllerContext.liveOrShuttingDownBrokerIds.toSeq, - Set(topicAndPartition)) + Set(TopicAndPartition(topic, partition))) } def addStopReplicaRequestForBrokers(brokerIds: Seq[Int], topic: String, partition: Int, deletePartition: Boolean, - callback: (RequestOrResponse, Int) => Unit = null) { + callback: (AbstractRequestResponse, Int) => Unit = null) { brokerIds.filter(b => b >= 0).foreach { brokerId => stopReplicaRequestMap.getOrElseUpdate(brokerId, Seq.empty[StopReplicaRequestInfo]) val v = stopReplicaRequestMap(brokerId) if(callback != null) stopReplicaRequestMap(brokerId) = v :+ StopReplicaRequestInfo(PartitionAndReplica(topic, partition, brokerId), - deletePartition, (r: RequestOrResponse) => { callback(r, brokerId) }) + deletePartition, (r: AbstractRequestResponse) => callback(r, brokerId)) else stopReplicaRequestMap(brokerId) = v :+ StopReplicaRequestInfo(PartitionAndReplica(topic, partition, brokerId), deletePartition) @@ -236,7 +298,7 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging /** 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) { + callback: AbstractRequestResponse => Unit = null) { def updateMetadataRequestMapFor(partition: TopicAndPartition, beingDeleted: Boolean) { val leaderIsrAndControllerEpochOpt = controllerContext.partitionLeadershipInfo.get(partition) leaderIsrAndControllerEpochOpt match { @@ -249,8 +311,8 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging PartitionStateInfo(leaderIsrAndControllerEpoch, replicas) } brokerIds.filter(b => b >= 0).foreach { brokerId => - updateMetadataRequestMap.getOrElseUpdate(brokerId, new mutable.HashMap[TopicAndPartition, PartitionStateInfo]) - updateMetadataRequestMap(brokerId).put(partition, partitionStateInfo) + updateMetadataRequestMap.getOrElseUpdate(brokerId, mutable.Map.empty[TopicPartition, PartitionStateInfo]) + updateMetadataRequestMap(brokerId).put(new TopicPartition(partition.topic, partition.partition), partitionStateInfo) } case None => info("Leader not yet assigned for partition %s. Skip sending UpdateMetadataRequest.".format(partition)) @@ -267,83 +329,143 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging else givenPartitions -- controller.deleteTopicManager.partitionsToBeDeleted } - filteredPartitions.foreach(partition => updateMetadataRequestMapFor(partition, beingDeleted = false)) + if (filteredPartitions.isEmpty) + brokerIds.filter(b => b >= 0).foreach { brokerId => + updateMetadataRequestMap.getOrElseUpdate(brokerId, mutable.Map.empty[TopicPartition, PartitionStateInfo]) + } + else + filteredPartitions.foreach(partition => updateMetadataRequestMapFor(partition, beingDeleted = false)) + controller.deleteTopicManager.partitionsToBeDeleted.foreach(partition => updateMetadataRequestMapFor(partition, beingDeleted = true)) } - def sendRequestsToBrokers(controllerEpoch: Int, correlationId: Int) { - leaderAndIsrRequestMap.foreach { m => - val broker = m._1 - val partitionStateInfos = m._2.toMap - val leaderIds = partitionStateInfos.map(_._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader).toSet - val leaders = controllerContext.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)) - val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfos, leaders, controllerId, controllerEpoch, correlationId, clientId) - for (p <- partitionStateInfos) { - val typeOfRequest = if (broker == p._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader) "become-leader" else "become-follower" - stateChangeLogger.trace(("Controller %d epoch %d sending %s LeaderAndIsr request %s with correlationId %d to broker %d " + - "for partition [%s,%d]").format(controllerId, controllerEpoch, typeOfRequest, - p._2.leaderIsrAndControllerEpoch, correlationId, broker, - p._1._1, p._1._2)) + def sendRequestsToBrokers(controllerEpoch: Int) { + try { + leaderAndIsrRequestMap.foreach { case (broker, partitionStateInfos) => + partitionStateInfos.foreach { case (topicPartition, state) => + val typeOfRequest = if (broker == state.leaderIsrAndControllerEpoch.leaderAndIsr.leader) "become-leader" else "become-follower" + stateChangeLogger.trace(("Controller %d epoch %d sending %s LeaderAndIsr request %s to broker %d " + + "for partition [%s,%d]").format(controllerId, controllerEpoch, typeOfRequest, + state.leaderIsrAndControllerEpoch, broker, + topicPartition.topic, topicPartition.partition)) + } + val leaderIds = partitionStateInfos.map(_._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader).toSet + val leaders = controllerContext.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)).map { b => + val brokerEndPoint = b.getBrokerEndPoint(controller.config.interBrokerSecurityProtocol) + new LeaderAndIsrRequest.EndPoint(brokerEndPoint.id, brokerEndPoint.host, brokerEndPoint.port) + } + val partitionStates = partitionStateInfos.map { case (topicPartition, partitionStateInfo) => + val LeaderIsrAndControllerEpoch(leaderIsr, controllerEpoch) = partitionStateInfo.leaderIsrAndControllerEpoch + val partitionState = new LeaderAndIsrRequest.PartitionState(controllerEpoch, leaderIsr.leader, + leaderIsr.leaderEpoch, leaderIsr.isr.map(Integer.valueOf).asJava, leaderIsr.zkVersion, + partitionStateInfo.allReplicas.map(Integer.valueOf).asJava + ) + topicPartition -> partitionState + } + val leaderAndIsrRequest = new LeaderAndIsrRequest(controllerId, controllerEpoch, partitionStates.asJava, leaders.asJava) + controller.sendRequest(broker, ApiKeys.LEADER_AND_ISR, None, leaderAndIsrRequest, null) } - controller.sendRequest(broker, leaderAndIsrRequest, null) - } - leaderAndIsrRequestMap.clear() - updateMetadataRequestMap.foreach { m => - val broker = m._1 - val partitionStateInfos = m._2.toMap - val updateMetadataRequest = new UpdateMetadataRequest(controllerId, controllerEpoch, correlationId, clientId, - partitionStateInfos, controllerContext.liveOrShuttingDownBrokers) - partitionStateInfos.foreach(p => stateChangeLogger.trace(("Controller %d epoch %d sending UpdateMetadata request %s with " + - "correlationId %d to broker %d for partition %s").format(controllerId, controllerEpoch, p._2.leaderIsrAndControllerEpoch, - 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 - 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" - .format(broker, stopReplicaWithoutDelete.mkString(","))) - replicaInfoList.foreach { r => - val stopReplicaRequest = new StopReplicaRequest(r.deletePartition, - Set(TopicAndPartition(r.replica.topic, r.replica.partition)), controllerId, controllerEpoch, correlationId) - controller.sendRequest(broker, stopReplicaRequest, r.callback) + leaderAndIsrRequestMap.clear() + updateMetadataRequestMap.foreach { case (broker, partitionStateInfos) => + + partitionStateInfos.foreach(p => stateChangeLogger.trace(("Controller %d epoch %d sending UpdateMetadata request %s " + + "to broker %d for partition %s").format(controllerId, controllerEpoch, p._2.leaderIsrAndControllerEpoch, + broker, p._1))) + val partitionStates = partitionStateInfos.map { case (topicPartition, partitionStateInfo) => + val LeaderIsrAndControllerEpoch(leaderIsr, controllerEpoch) = partitionStateInfo.leaderIsrAndControllerEpoch + val partitionState = new UpdateMetadataRequest.PartitionState(controllerEpoch, leaderIsr.leader, + leaderIsr.leaderEpoch, leaderIsr.isr.map(Integer.valueOf).asJava, leaderIsr.zkVersion, + partitionStateInfo.allReplicas.map(Integer.valueOf).asJava + ) + topicPartition -> partitionState + } + + val version = if (controller.config.interBrokerProtocolVersion.onOrAfter(KAFKA_090)) (1: Short) else (0: Short) + + val updateMetadataRequest = + if (version == 0) { + val liveBrokers = controllerContext.liveOrShuttingDownBrokers.map { broker => + val brokerEndPoint = broker.getBrokerEndPoint(SecurityProtocol.PLAINTEXT) + new UpdateMetadataRequest.BrokerEndPoint(brokerEndPoint.id, brokerEndPoint.host, brokerEndPoint.port) + } + new UpdateMetadataRequest(controllerId, controllerEpoch, liveBrokers.asJava, partitionStates.asJava) + } + else { + val liveBrokers = controllerContext.liveOrShuttingDownBrokers.map { broker => + val endPoints = broker.endPoints.map { case (securityProtocol, endPoint) => + securityProtocol -> new UpdateMetadataRequest.EndPoint(endPoint.host, endPoint.port) + } + new UpdateMetadataRequest.Broker(broker.id, endPoints.asJava) + } + new UpdateMetadataRequest(controllerId, controllerEpoch, partitionStates.asJava, liveBrokers.asJava) + } + + controller.sendRequest(broker, ApiKeys.UPDATE_METADATA_KEY, Some(version), updateMetadataRequest, null) + } + updateMetadataRequestMap.clear() + stopReplicaRequestMap.foreach { case (broker, replicaInfoList) => + 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" + .format(broker, stopReplicaWithoutDelete.mkString(","))) + replicaInfoList.foreach { r => + val stopReplicaRequest = new StopReplicaRequest(controllerId, controllerEpoch, r.deletePartition, + Set(new TopicPartition(r.replica.topic, r.replica.partition)).asJava) + controller.sendRequest(broker, ApiKeys.STOP_REPLICA, None, stopReplicaRequest, r.callback) + } + } + stopReplicaRequestMap.clear() + } catch { + case e : Throwable => { + if (leaderAndIsrRequestMap.size > 0) { + error("Haven't been able to send leader and isr requests, current state of " + + s"the map is $leaderAndIsrRequestMap") + } + if (updateMetadataRequestMap.size > 0) { + error("Haven't been able to send metadata update requests, current state of " + + s"the map is $updateMetadataRequestMap") + } + if (stopReplicaRequestMap.size > 0) { + error("Haven't been able to send stop replica requests, current state of " + + s"the map is $stopReplicaRequestMap") + } + throw new IllegalStateException(e) } } - stopReplicaRequestMap.clear() } } -case class ControllerBrokerStateInfo(channel: BlockingChannel, +case class ControllerBrokerStateInfo(networkClient: NetworkClient, + brokerNode: Node, broker: Broker, - messageQueue: BlockingQueue[(RequestOrResponse, (RequestOrResponse) => Unit)], + messageQueue: BlockingQueue[QueueItem], requestSendThread: RequestSendThread) -case class StopReplicaRequestInfo(replica: PartitionAndReplica, deletePartition: Boolean, callback: (RequestOrResponse) => Unit = null) +case class StopReplicaRequestInfo(replica: PartitionAndReplica, deletePartition: Boolean, callback: AbstractRequestResponse => Unit = null) -class Callbacks private (var leaderAndIsrResponseCallback:(RequestOrResponse) => Unit = null, - var updateMetadataResponseCallback:(RequestOrResponse) => Unit = null, - var stopReplicaResponseCallback:(RequestOrResponse, Int) => Unit = null) +class Callbacks private (var leaderAndIsrResponseCallback: AbstractRequestResponse => Unit = null, + var updateMetadataResponseCallback: AbstractRequestResponse => Unit = null, + var stopReplicaResponseCallback: (AbstractRequestResponse, Int) => Unit = null) object Callbacks { class CallbackBuilder { - var leaderAndIsrResponseCbk:(RequestOrResponse) => Unit = null - var updateMetadataResponseCbk:(RequestOrResponse) => Unit = null - var stopReplicaResponseCbk:(RequestOrResponse, Int) => Unit = null + var leaderAndIsrResponseCbk: AbstractRequestResponse => Unit = null + var updateMetadataResponseCbk: AbstractRequestResponse => Unit = null + var stopReplicaResponseCbk: (AbstractRequestResponse, Int) => Unit = null - def leaderAndIsrCallback(cbk: (RequestOrResponse) => Unit): CallbackBuilder = { + def leaderAndIsrCallback(cbk: AbstractRequestResponse => Unit): CallbackBuilder = { leaderAndIsrResponseCbk = cbk this } - def updateMetadataCallback(cbk: (RequestOrResponse) => Unit): CallbackBuilder = { + def updateMetadataCallback(cbk: AbstractRequestResponse => Unit): CallbackBuilder = { updateMetadataResponseCbk = cbk this } - def stopReplicaCallback(cbk: (RequestOrResponse, Int) => Unit): CallbackBuilder = { + def stopReplicaCallback(cbk: (AbstractRequestResponse, Int) => Unit): CallbackBuilder = { stopReplicaResponseCbk = cbk this } 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 8ab4a1b8072c9..a7b44cab501a9 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -16,10 +16,13 @@ */ package kafka.controller -import collection._ -import collection.Set +import java.util + +import org.apache.kafka.common.protocol.ApiKeys +import org.apache.kafka.common.requests.{AbstractRequest, AbstractRequestResponse} + +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 @@ -30,18 +33,18 @@ import kafka.log.LogConfig import kafka.metrics.{KafkaTimer, KafkaMetricsGroup} 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.apache.kafka.common.metrics.Metrics +import org.apache.kafka.common.utils.Time +import org.I0Itec.zkclient.{IZkChildListener, IZkDataListener, IZkStateListener, ZkClient, ZkConnection} import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException} -import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.locks.ReentrantLock -import scala.None import kafka.server._ -import scala.Some import kafka.common.TopicAndPartition class ControllerContext(val zkClient: ZkClient, + val zkConnection: ZkConnection, val zkSessionTimeout: Int) { var controllerChannelManager: ControllerChannelManager = null val controllerLock: ReentrantLock = new ReentrantLock() @@ -49,12 +52,11 @@ class ControllerContext(val zkClient: ZkClient, val brokerShutdownLock: Object = new Object var epoch: Int = KafkaController.InitialControllerEpoch - 1 var epochZkVersion: Int = KafkaController.InitialControllerEpochZkVersion - 1 - val correlationId: AtomicInteger = new AtomicInteger(0) var allTopics: Set[String] = Set.empty var partitionReplicaAssignment: mutable.Map[TopicAndPartition, Seq[Int]] = mutable.Map.empty var partitionLeadershipInfo: mutable.Map[TopicAndPartition, LeaderIsrAndControllerEpoch] = mutable.Map.empty - var partitionsBeingReassigned: mutable.Map[TopicAndPartition, ReassignedPartitionsContext] = new mutable.HashMap - var partitionsUndergoingPreferredReplicaElection: mutable.Set[TopicAndPartition] = new mutable.HashSet + val partitionsBeingReassigned: mutable.Map[TopicAndPartition, ReassignedPartitionsContext] = new mutable.HashMap + val partitionsUndergoingPreferredReplicaElection: mutable.Set[TopicAndPartition] = new mutable.HashSet private var liveBrokersUnderlying: Set[Broker] = Set.empty private var liveBrokerIdsUnderlying: Set[Int] = Set.empty @@ -115,10 +117,11 @@ 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 } + } @@ -151,11 +154,11 @@ object KafkaController extends Logging { } } -class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerState: BrokerState) extends Logging with KafkaMetricsGroup { +class KafkaController(val config : KafkaConfig, zkClient: ZkClient, zkConnection: ZkConnection, val brokerState: BrokerState, time: Time, metrics: Metrics, threadNamePrefix: Option[String] = None) extends Logging with KafkaMetricsGroup { this.logIdent = "[Controller " + config.brokerId + "]: " private var isRunning = true private val stateChangeLogger = KafkaController.stateChangeLogger - val controllerContext = new ControllerContext(zkClient, config.zkSessionTimeoutMs) + val controllerContext = new ControllerContext(zkClient, zkConnection, config.zkSessionTimeoutMs) val partitionStateMachine = new PartitionStateMachine(this) val replicaStateMachine = new ReplicaStateMachine(this) private val controllerElector = new ZookeeperLeaderElector(controllerContext, ZkUtils.ControllerPath, onControllerFailover, @@ -172,6 +175,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt private val partitionReassignedListener = new PartitionsReassignedListener(this) private val preferredReplicaElectionListener = new PreferredReplicaElectionListener(this) + private val isrChangeNotificationListener = new IsrChangeNotificationListener(this) newGauge( "ActiveControllerCount", @@ -212,7 +216,11 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt 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 @@ -260,11 +268,21 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt } 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) - + try { + brokerRequestBatch.newBatch() + brokerRequestBatch.addStopReplicaRequestForBrokers(Seq(id), topicAndPartition.topic, + topicAndPartition.partition, deletePartition = false) + brokerRequestBatch.sendRequestsToBrokers(epoch) + } catch { + case e : IllegalStateException => { + // Resign if the controller is in an illegal state + error("Forcing the controller to resign") + brokerRequestBatch.clear() + controllerElector.resign() + + throw e + } + } // 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) @@ -306,6 +324,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt incrementControllerEpoch(zkClient) // before reading source of truth from zookeeper, register the listeners to get broker/topic callbacks registerReassignedPartitionsListener() + registerIsrChangeNotificationListener() registerPreferredReplicaElectionListener() partitionStateMachine.registerListeners() replicaStateMachine.registerListeners() @@ -324,7 +343,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt 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() } @@ -337,7 +356,9 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt * required to clean up internal controller data structures */ def onControllerResignation() { + debug("Controller resigning, broker id %d".format(config.brokerId)) // de-register listeners + deregisterIsrChangeNotificationListener() deregisterReassignedPartitionsListener() deregisterPreferredReplicaElectionListener() @@ -345,12 +366,13 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt if (deleteTopicManager != null) deleteTopicManager.shutdown() + // 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 leader rebalance scheduler - if (config.autoLeaderRebalanceEnable) - autoRebalanceScheduler.shutdown() // shutdown partition state machine partitionStateMachine.shutdown() // shutdown replica state machine @@ -364,6 +386,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt controllerContext.epoch=0 controllerContext.epochZkVersion=0 brokerState.newState(RunningAsBroker) + + info("Broker %d resigned as the controller".format(config.brokerId)) } } @@ -379,8 +403,9 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt /** * 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: @@ -392,10 +417,11 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt 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) @@ -425,6 +451,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt * 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 @@ -456,6 +483,12 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt // 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) + } } /** @@ -643,7 +676,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt */ def startup() = { inLock(controllerContext.controllerLock) { - info("Controller starting up"); + info("Controller starting up") registerSessionExpirationListener() isRunning = true controllerElector.startup @@ -663,8 +696,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt onControllerResignation() } - def sendRequest(brokerId : Int, request : RequestOrResponse, callback: (RequestOrResponse) => Unit = null) = { - controllerContext.controllerChannelManager.sendRequest(brokerId, request, callback) + def sendRequest(brokerId: Int, apiKey: ApiKeys, apiVersion: Option[Short], request: AbstractRequest, callback: AbstractRequestResponse => Unit = null) = { + controllerContext.controllerChannelManager.sendRequest(brokerId, apiKey, apiVersion, request, callback) } def incrementControllerEpoch(zkClient: ZkClient) = { @@ -784,12 +817,12 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt } private def startChannelManager() { - controllerContext.controllerChannelManager = new ControllerChannelManager(controllerContext, config) + controllerContext.controllerChannelManager = new ControllerChannelManager(controllerContext, config, time, metrics, threadNamePrefix) 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) } @@ -871,9 +904,20 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt brokerRequestBatch.newBatch() updateLeaderEpoch(topicAndPartition.topic, topicAndPartition.partition) match { case Some(updatedLeaderIsrAndControllerEpoch) => - brokerRequestBatch.addLeaderAndIsrRequestForBrokers(replicasToReceiveRequest, topicAndPartition.topic, - topicAndPartition.partition, updatedLeaderIsrAndControllerEpoch, newAssignedReplicas) - brokerRequestBatch.sendRequestsToBrokers(controllerContext.epoch, controllerContext.correlationId.getAndIncrement) + try { + brokerRequestBatch.addLeaderAndIsrRequestForBrokers(replicasToReceiveRequest, topicAndPartition.topic, + topicAndPartition.partition, updatedLeaderIsrAndControllerEpoch, newAssignedReplicas) + brokerRequestBatch.sendRequestsToBrokers(controllerContext.epoch) + } catch { + case e : IllegalStateException => { + // Resign if the controller is in an illegal state + error("Forcing the controller to resign") + brokerRequestBatch.clear() + controllerElector.resign() + + throw e + } + } stateChangeLogger.trace(("Controller %d epoch %d sent LeaderAndIsr request %s with new assigned replica list %s " + "to leader %d for partition being reassigned %s").format(config.brokerId, controllerContext.epoch, updatedLeaderIsrAndControllerEpoch, newAssignedReplicas.mkString(","), updatedLeaderIsrAndControllerEpoch.leaderAndIsr.leader, topicAndPartition)) @@ -884,6 +928,16 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt } } + private def registerIsrChangeNotificationListener() = { + debug("Registering IsrChangeNotificationListener") + zkClient.subscribeChildChanges(ZkUtils.IsrChangeNotificationPath, isrChangeNotificationListener) + } + + private def deregisterIsrChangeNotificationListener() = { + debug("De-registering IsrChangeNotificationListener") + zkClient.unsubscribeChildChanges(ZkUtils.IsrChangeNotificationPath, isrChangeNotificationListener) + } + private def registerReassignedPartitionsListener() = { zkClient.subscribeDataChanges(ZkUtils.ReassignPartitionsPath, partitionReassignedListener) } @@ -970,9 +1024,20 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt * @param brokers The brokers that the update metadata request should be sent to */ def sendUpdateMetadataRequest(brokers: Seq[Int], partitions: Set[TopicAndPartition] = Set.empty[TopicAndPartition]) { - brokerRequestBatch.newBatch() - brokerRequestBatch.addUpdateMetadataRequestForBrokers(brokers, partitions) - brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement) + try { + brokerRequestBatch.newBatch() + brokerRequestBatch.addUpdateMetadataRequestForBrokers(brokers, partitions) + brokerRequestBatch.sendRequestsToBrokers(epoch) + } catch { + case e : IllegalStateException => { + // Resign if the controller is in an illegal state + error("Forcing the controller to resign") + brokerRequestBatch.clear() + controllerElector.resign() + + throw e + } + } } /** @@ -1009,8 +1074,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt // 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.props.props, AdminUtils.fetchTopicConfig(zkClient, - topicAndPartition.topic)).uncleanLeaderElectionEnable) { + if (newIsr.isEmpty && !LogConfig.fromProps(config.originals, AdminUtils.fetchEntityConfig(zkClient, + ConfigType.Topic, topicAndPartition.topic)).uncleanLeaderElectionEnable) { info("Retaining last ISR %d of partition %s since unclean leader election is disabled".format(replicaId, topicAndPartition)) newIsr = leaderAndIsr.isr } @@ -1108,6 +1173,10 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt controllerElector.elect } } + + override def handleSessionEstablishmentError(error: Throwable): Unit = { + //no-op handleSessionEstablishmentError in KafkaHealthCheck should handle this error in its handleSessionEstablishmentError + } } private def checkAndTriggerPartitionRebalance(): Unit = { @@ -1272,6 +1341,69 @@ class ReassignedPartitionsIsrChangeListener(controller: KafkaController, topic: } } +/** + * Called when leader intimates of isr change + * @param controller + */ +class IsrChangeNotificationListener(controller: KafkaController) extends IZkChildListener with Logging { + + 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 + try { + val topicAndPartitions: immutable.Set[TopicAndPartition] = childrenAsScala.map(x => getTopicAndPartition(x)).flatten.toSet + controller.updateLeaderAndIsrCache(topicAndPartitions) + processUpdateNotifications(topicAndPartitions) + } finally { + // delete processed children + childrenAsScala.map(x => ZkUtils.deletePath(controller.controllerContext.zkClient, + ZkUtils.IsrChangeNotificationPath + "/" + x)) + } + } + } + + private def processUpdateNotifications(topicAndPartitions: immutable.Set[TopicAndPartition]) { + val liveBrokers: Seq[Int] = controller.controllerContext.liveOrShuttingDownBrokerIds.toSeq + debug("Sending MetadataRequest to Brokers:" + liveBrokers + " for TopicAndPartitions:" + topicAndPartitions) + controller.sendUpdateMetadataRequest(liveBrokers, topicAndPartitions) + } + + private def getTopicAndPartition(child: String): Set[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 topicAndPartitions: mutable.Set[TopicAndPartition] = new mutable.HashSet[TopicAndPartition]() + val isrChanges = m.asInstanceOf[Map[String, Any]] + val topicAndPartitionList = isrChanges("partitions").asInstanceOf[List[Any]] + topicAndPartitionList.foreach { + case tp => + val topicAndPartition = tp.asInstanceOf[Map[String, Any]] + val topic = topicAndPartition("topic").asInstanceOf[String] + val partition = topicAndPartition("partition").asInstanceOf[Int] + topicAndPartitions += TopicAndPartition(topic, partition) + } + topicAndPartitions + case None => + error("Invalid topic and partition JSON: " + jsonOpt.get + " in ZK: " + changeZnode) + Set.empty + } + } else { + Set.empty + } + } +} + +object IsrChangeNotificationListener { + val version: Long = 1L +} + /** * Starts the preferred replica leader election for the list of partitions specified under * /admin/preferred_replica_election - @@ -1300,8 +1432,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) } } @@ -1323,7 +1454,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 4a31c7271c2d0..4ebeb5a0ef3d2 100644 --- a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala +++ b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala @@ -21,7 +21,7 @@ import kafka.api.LeaderAndIsr import kafka.log.LogConfig import kafka.utils.Logging import kafka.common.{LeaderElectionNotNeededException, TopicAndPartition, StateChangeFailedException, NoReplicaOnlineException} -import kafka.server.KafkaConfig +import kafka.server.{ConfigType, KafkaConfig} trait PartitionLeaderSelector { @@ -61,8 +61,8 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext, confi 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.props.props, AdminUtils.fetchTopicConfig(controllerContext.zkClient, - topicAndPartition.topic)).uncleanLeaderElectionEnable) { + if (!LogConfig.fromProps(config.originals, AdminUtils.fetchEntityConfig(controllerContext.zkClient, + ConfigType.Topic, 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(","))) 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 e20b63a6ec1c1..675a8071beb22 --- a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala +++ b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala @@ -25,9 +25,8 @@ import kafka.common.{LeaderElectionNotNeededException, TopicAndPartition, StateC 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 @@ -120,7 +119,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { handleStateChange(topicAndPartition.topic, topicAndPartition.partition, OnlinePartition, controller.offlinePartitionSelector, (new CallbackBuilder).build) } - brokerRequestBatch.sendRequestsToBrokers(controller.epoch, controllerContext.correlationId.getAndIncrement) + brokerRequestBatch.sendRequestsToBrokers(controller.epoch) } catch { case e: Throwable => error("Error while moving some partitions to the online state", e) // TODO: It is not enough to bail out and log an error, it is important to trigger leader election for those partitions @@ -145,7 +144,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { partitions.foreach { topicAndPartition => handleStateChange(topicAndPartition.topic, topicAndPartition.partition, targetState, leaderSelector, callbacks) } - brokerRequestBatch.sendRequestsToBrokers(controller.epoch, controllerContext.correlationId.getAndIncrement) + brokerRequestBatch.sendRequestsToBrokers(controller.epoch) }catch { case e: Throwable => error("Error while moving some partitions to %s state".format(targetState), e) // TODO: It is not enough to bail out and log an error, it is important to trigger state changes for those partitions @@ -189,7 +188,6 @@ 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 %s to %s with assigned replicas %s" @@ -266,20 +264,9 @@ 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 + * a leader and isr path in zookeeper. Once the partition moves to the OnlinePartition state, its leader and isr * path gets initialized and it never goes back to the NewPartition state. From here, it can only go to the * OfflinePartition state. * @param topicAndPartition The topic/partition whose leader and isr path is to be initialized @@ -484,8 +471,6 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { topicsToBeDeleted --= nonExistentTopics if(topicsToBeDeleted.size > 0) { info("Starting topic deletion for topics " + topicsToBeDeleted.mkString(",")) - // add topic to deletion list - controller.deleteTopicManager.enqueueTopicsForDeletion(topicsToBeDeleted) // mark topic ineligible for deletion if other state changes are in progress topicsToBeDeleted.foreach { topic => val preferredReplicaElectionInProgress = @@ -495,6 +480,8 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { if(preferredReplicaElectionInProgress || partitionReassignmentInProgress) controller.deleteTopicManager.markTopicIneligibleForDeletion(Set(topic)) } + // add topic to deletion list + controller.deleteTopicManager.enqueueTopicsForDeletion(topicsToBeDeleted) } } } @@ -527,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) } } 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 3e87e1d36f87b..acad83a3327ea --- a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala +++ b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala @@ -24,7 +24,7 @@ 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 @@ -112,7 +112,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { try { brokerRequestBatch.newBatch() replicas.foreach(r => handleStateChange(r, targetState, callbacks)) - brokerRequestBatch.sendRequestsToBrokers(controller.epoch, controllerContext.correlationId.getAndIncrement) + brokerRequestBatch.sendRequestsToBrokers(controller.epoch) }catch { case e: Throwable => error("Error while moving some replicas to %s state".format(targetState), e) } @@ -282,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 = { 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 219c4136e905a..9e39dd5776eca --- a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala +++ b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala @@ -16,12 +16,17 @@ */ package kafka.controller + +import kafka.server.ConfigType +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests.{StopReplicaResponse, AbstractRequestResponse} + import collection.mutable +import collection.JavaConverters._ 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 kafka.common.TopicAndPartition import java.util.concurrent.locks.ReentrantLock import java.util.concurrent.atomic.AtomicBoolean @@ -89,7 +94,8 @@ class TopicDeletionManager(controller: KafkaController, def start() { if (isDeleteTopicEnabled) { deleteTopicsThread = new DeleteTopicsThread() - deleteTopicStateChanged.set(true) + if (topicsToBeDeleted.size > 0) + deleteTopicStateChanged.set(true) deleteTopicsThread.start() } } @@ -214,7 +220,7 @@ class TopicDeletionManager(controller: KafkaController, */ private def awaitTopicDeletionNotification() { inLock(deleteLock) { - while(!deleteTopicsThread.isRunning.get() && !deleteTopicStateChanged.compareAndSet(true, false)) { + while(deleteTopicsThread.isRunning.get() && !deleteTopicStateChanged.compareAndSet(true, false)) { debug("Waiting for signal to start or continue topic deletion") deleteTopicsCond.await() } @@ -283,7 +289,7 @@ class TopicDeletionManager(controller: KafkaController, topicsToBeDeleted -= topic partitionsToBeDeleted.retain(_.topic != topic) controllerContext.zkClient.deleteRecursive(ZkUtils.getTopicPath(topic)) - controllerContext.zkClient.deleteRecursive(ZkUtils.getTopicConfigPath(topic)) + controllerContext.zkClient.deleteRecursive(ZkUtils.getEntityConfigPath(ConfigType.Topic, topic)) controllerContext.zkClient.delete(ZkUtils.getDeleteTopicPath(topic)) controllerContext.removeTopic(topic) } @@ -359,20 +365,20 @@ class TopicDeletionManager(controller: KafkaController, startReplicaDeletion(replicasPerPartition) } - private def deleteTopicStopReplicaCallback(stopReplicaResponseObj: RequestOrResponse, replicaId: Int) { + private def deleteTopicStopReplicaCallback(stopReplicaResponseObj: AbstractRequestResponse, replicaId: Int) { val stopReplicaResponse = stopReplicaResponseObj.asInstanceOf[StopReplicaResponse] debug("Delete topic callback invoked for %s".format(stopReplicaResponse)) - val partitionsInError = if(stopReplicaResponse.errorCode != ErrorMapping.NoError) { - stopReplicaResponse.responseMap.keySet - } else - stopReplicaResponse.responseMap.filter(p => p._2 != ErrorMapping.NoError).map(_._1).toSet + val responseMap = stopReplicaResponse.responses.asScala + val partitionsInError = + if (stopReplicaResponse.errorCode != Errors.NONE.code) responseMap.keySet + else responseMap.filter { case (_, error) => error != Errors.NONE.code }.map(_._1).toSet val replicasInError = partitionsInError.map(p => PartitionAndReplica(p.topic, p.partition, replicaId)) inLock(controllerContext.controllerLock) { // move all the failed replicas to ReplicaDeletionIneligible failReplicaDeletion(replicasInError) - if(replicasInError.size != stopReplicaResponse.responseMap.size) { + if (replicasInError.size != responseMap.size) { // some replicas could have been successfully deleted - val deletedReplicas = stopReplicaResponse.responseMap.keySet -- partitionsInError + val deletedReplicas = responseMap.keySet -- partitionsInError completeReplicaDeletion(deletedReplicas.map(p => PartitionAndReplica(p.topic, p.partition, replicaId))) } } @@ -429,4 +435,3 @@ class TopicDeletionManager(controller: KafkaController, } } } - 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..2cdab85b79bb5 --- /dev/null +++ b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala @@ -0,0 +1,497 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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) { + responseCallback(Errors.ILLEGAL_GENERATION.code) + } else if (!group.is(Stable)) { + responseCallback(Errors.REBALANCE_IN_PROGRESS.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 (generationId < 0) + // the group is not relying on Kafka for partition management, so allow the commit + offsetManager.storeOffsets(groupId, consumerId, generationId, offsetMetadata, responseCallback) + else + // the group has failed over to this coordinator (which will be handled in KAFKA-2017), + // or this is a request coming from an older generation. either way, reject the commit + responseCallback(offsetMetadata.mapValues(_ => Errors.ILLEGAL_GENERATION.code)) + } 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 ++ 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 index 1b28861cdf7df..4345a8e80914d 100644 --- a/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala +++ b/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala @@ -17,13 +17,14 @@ package kafka.javaapi -import kafka.cluster.Broker +import java.nio.ByteBuffer +import kafka.cluster.BrokerEndPoint class ConsumerMetadataResponse(private val underlying: kafka.api.ConsumerMetadataResponse) { def errorCode = underlying.errorCode - def coordinator: Broker = { + def coordinator: BrokerEndPoint = { import kafka.javaapi.Implicits._ underlying.coordinatorOpt } @@ -40,3 +41,7 @@ class ConsumerMetadataResponse(private val underlying: kafka.api.ConsumerMetadat 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 27fc1eb9d578e..456c3c423a1dd 100644 --- a/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala +++ b/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala @@ -22,7 +22,8 @@ import kafka.common.{OffsetAndMetadata, TopicAndPartition} class OffsetCommitRequest(groupId: String, requestInfo: java.util.Map[TopicAndPartition, OffsetAndMetadata], correlationId: Int, - clientId: String) { + clientId: String, + versionId: Short) { val underlying = { val scalaMap: collection.immutable.Map[TopicAndPartition, OffsetAndMetadata] = { import collection.JavaConversions._ @@ -32,11 +33,21 @@ class OffsetCommitRequest(groupId: String, kafka.api.OffsetCommitRequest( groupId = groupId, requestInfo = scalaMap, + versionId = versionId, correlationId = correlationId, clientId = clientId ) } + 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 c2d3d114b82a8..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 @@ -27,5 +29,12 @@ class OffsetCommitResponse(private val underlying: kafka.api.OffsetCommitRespons 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 f384e04678df1..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 { @@ -52,17 +52,17 @@ class TopicMetadata(private val underlying: kafka.api.TopicMetadata) { 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 } diff --git a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala b/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala index b0b7be14d494a..568d0ac0b2ade 100644 --- a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala +++ b/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala @@ -16,12 +16,11 @@ */ 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, val correlationId: Int, diff --git a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java b/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java index cc3400ff81fc0..444cd1d4b34e6 100644 --- a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java +++ b/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java @@ -20,6 +20,9 @@ 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; @@ -64,6 +67,20 @@ public interface ConsumerConnector { 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); + + /** + * Wire in a consumer rebalance listener to be executed when consumer rebalance occurs. + * @param listener The consumer rebalance listener to wire in + */ + public void setConsumerRebalanceListener(ConsumerRebalanceListener listener); + /** * 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 1f98db5d692ad..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,8 @@ package kafka.javaapi.consumer import kafka.serializer._ import kafka.consumer._ -import kafka.common.MessageStreamsExistException -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 /** @@ -108,13 +107,21 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, 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() { underlying.shutdown } diff --git a/core/src/main/scala/kafka/javaapi/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/javaapi/message/ByteBufferMessageSet.scala index 0125565c84aea..df3027958af23 100644 --- a/core/src/main/scala/kafka/javaapi/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/javaapi/message/ByteBufferMessageSet.scala @@ -19,14 +19,14 @@ package kafka.javaapi.message import java.util.concurrent.atomic.AtomicLong import java.nio.ByteBuffer import kafka.message._ -import kafka.javaapi.Implicits.javaListToScalaBuffer + +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]) { diff --git a/core/src/main/scala/kafka/log/CleanerConfig.scala b/core/src/main/scala/kafka/log/CleanerConfig.scala index ade838672d671..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 = 15 * 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 b2652ddbe2f85..949dc0253781a --- a/core/src/main/scala/kafka/log/FileMessageSet.scala +++ b/core/src/main/scala/kafka/log/FileMessageSet.scala @@ -27,6 +27,7 @@ import kafka.message._ import kafka.common.KafkaException import java.util.concurrent.TimeUnit import kafka.metrics.{KafkaTimer, KafkaMetricsGroup} +import org.apache.kafka.common.network.TransportLayer /** * An on-disk message set. An optional start and end position can be applied to the message set @@ -54,7 +55,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 +67,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 @@ -144,7 +158,12 @@ class FileMessageSet private[kafka](@volatile var file: File, throw new KafkaException("Size of FileMessageSet %s has been truncated during write: old size %d, new size %d" .format(file.getAbsolutePath, _size.get(), newSize)) } - val bytesTransferred = channel.transferTo(start + writePosition, math.min(size, sizeInBytes), destChannel).toInt + val position = start + writePosition + val count = math.min(size, sizeInBytes) + val bytesTransferred = (destChannel match { + case tl: TransportLayer => tl.transferFrom(channel, position, count) + case dc => channel.transferTo(position, count, dc) + }).toInt trace("FileMessageSet " + file.getAbsolutePath + " : bytes transferred : " + bytesTransferred + " bytes requested for transfer : " + math.min(size, sizeInBytes)) bytesTransferred @@ -223,15 +242,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() } @@ -272,6 +299,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 0ddf97bd30311..02205c914b9f3 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -31,6 +31,22 @@ 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. @@ -62,6 +78,13 @@ class Log(val dir: File, /* 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() @@ -69,21 +92,35 @@ class Log(val dir: File, /* Calculate the offset of the next message */ @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 + "-" + "LogStartOffset", - new Gauge[Long] { def value = logStartOffset }) + newGauge("LogStartOffset", + new Gauge[Long] { + def value = logStartOffset + }, + tags) - newGauge(name + "-" + "LogEndOffset", - new Gauge[Long] { def value = logEndOffset }) - - newGauge(name + "-" + "Size", - new Gauge[Long] {def value = size}) + 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() @@ -92,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) @@ -104,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 } } } @@ -136,36 +169,71 @@ 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) { @@ -252,14 +320,11 @@ class Log(val dir: File, lock synchronized { appendInfo.firstOffset = nextOffsetMetadata.messageOffset - // maybe roll the log if this segment is full - val segment = maybeRoll() - if(assignOffsets) { // 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) } @@ -282,6 +347,16 @@ class Log(val dir: File, } } + // 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) @@ -301,17 +376,6 @@ 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 shallowCount The number of shallow messages - * @param validBytes The number of valid bytes - * @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, validBytes: Int, offsetsMonotonic: Boolean) - /** * Validate the following: *

              @@ -333,7 +397,7 @@ class Log(val dir: File, 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 @@ -361,14 +425,18 @@ class Log(val dir: File, shallowMessageCount += 1 validBytesCount += messageSize - + val messageCodec = m.compressionCodec if(messageCodec != NoCompressionCodec) - codec = messageCodec + sourceCodec = messageCodec } - LogAppendInfo(firstOffset, lastOffset, codec, shallowMessageCount, validBytesCount, 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 @@ -402,22 +470,41 @@ class Log(val dir: File, 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 = nextOffsetMetadata.messageOffset + // Because we don't use lock for reading, the synchronization is a little bit tricky. + // We create the local variables to avoid race conditions with updates to the log. + val currentNextOffsetMetadata = nextOffsetMetadata + val next = currentNextOffsetMetadata.messageOffset if(startOffset == next) - return FetchDataInfo(nextOffsetMetadata, MessageSet.Empty) - + return FetchDataInfo(currentNextOffsetMetadata, MessageSet.Empty) + var entry = segments.floorEntry(startOffset) - + // attempt to read beyond the log end offset is an error if(startOffset > next || entry == null) throw new OffsetOutOfRangeException("Request for offset %d but we only have log segments in the range %d to %d.".format(startOffset, segments.firstKey, next)) - // do the read on the segment with a base offset less than the target offset + // Do the read on the segment with a base offset less than the target offset // but if that segment doesn't contain any messages with an offset greater than that // continue to read from successive segments until we get some messages or we reach the end of the log while(entry != null) { - val fetchInfo = entry.getValue.read(startOffset, maxOffset, maxLength) + // If the fetch occurs on the active segment, there might be a race condition where two fetch requests occur after + // the message is appended but before the nextOffsetMetadata is updated. In that case the second fetch may + // cause OffsetOutOfRangeException. To solve that, we cap the reading up to exposed position instead of the log + // end of the active segment. + val maxPosition = { + if (entry == segments.lastEntry) { + val exposedPos = nextOffsetMetadata.relativePositionInSegment.toLong + // Check the segment again in case a new segment has just rolled out. + if (entry != segments.lastEntry) + // New log segment has rolled out, we can read up to the file end. + entry.getValue.size + else + exposedPos + } else { + entry.getValue.size + } + } + val fetchInfo = entry.getValue.read(startOffset, maxOffset, maxLength, maxPosition) if(fetchInfo == null) { entry = segments.higherEntry(entry.getKey) } else { @@ -438,7 +525,7 @@ class Log(val dir: File, def convertToOffsetMetadata(offset: Long): LogOffsetMetadata = { try { val fetchDataInfo = read(offset, 1) - fetchDataInfo.fetchOffset + fetchDataInfo.fetchOffsetMetadata } catch { case e: OffsetOutOfRangeException => LogOffsetMetadata.UnknownOffsetMetadata } @@ -489,13 +576,21 @@ class Log(val dir: File, 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, @@ -504,7 +599,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 @@ -529,22 +624,31 @@ 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)) - + // We need to update the segment base offset and append position data of the metadata when log rolls. + // The next offset should not change. + updateLogEndOffset(nextOffsetMetadata.messageOffset) // schedule an asynchronous flush of the old segment scheduler.schedule("flush-log", () => flush(newOffset), delay = 0L) info("Rolled new log segment for '" + name + "' in %.0f ms.".format((System.nanoTime - start) / (1000.0*1000.0))) - + segment } } @@ -583,9 +687,10 @@ class Log(val dir: File, */ private[log] def delete() { lock synchronized { + removeLogMetrics() logSegments.foreach(_.delete()) segments.clear() - Utils.rm(dir) + CoreUtils.rm(dir) } } @@ -627,7 +732,11 @@ class Log(val dir: File, newOffset, indexIntervalBytes = config.indexInterval, maxIndexSize = config.maxIndexSize, - time = time)) + rollJitterMs = config.randomSegmentJitter, + time = time, + fileAlreadyExists = false, + initFileSize = initFileSize, + preallocate = config.preallocate)) updateLogEndOffset(newOffset) this.recoveryPoint = math.min(newOffset, this.recoveryPoint) } @@ -705,14 +814,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 @@ -727,7 +854,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 @@ -795,9 +931,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 c20de4ad4734c..b36ea0dd7f954 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -17,21 +17,18 @@ package kafka.log +import java.io.{DataOutputStream, File} +import java.nio._ +import java.util.Date +import java.util.concurrent.{CountDownLatch, TimeUnit} + +import com.yammer.metrics.core.Gauge import kafka.common._ import kafka.message._ -import kafka.utils._ import kafka.metrics.KafkaMetricsGroup +import kafka.utils._ import scala.collection._ -import scala.math -import java.nio._ -import java.util.Date -import java.io.File -import java.lang.IllegalStateException -import java.util.concurrent.CountDownLatch -import java.util.concurrent.TimeUnit - -import com.yammer.metrics.core.Gauge /** * The cleaner is responsible for removing obsolete records from logs which have the dedupe retention strategy. @@ -71,8 +68,8 @@ class LogCleaner(val config: CleanerConfig, val logs: Pool[TopicAndPartition, Log], time: Time = SystemTime) extends Logging with KafkaMetricsGroup { - /* for managing the state of partitions being cleaned. */ - private val cleanerManager = new LogCleanerManager(logDirs, logs); + /* 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, @@ -129,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. @@ -145,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 = { @@ -236,7 +239,7 @@ class LogCleaner(val config: CleanerConfig, "\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, @@ -246,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)) + } } } @@ -338,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 @@ -367,7 +373,7 @@ private[log] class Cleaner(val id: Int, } catch { case e: LogCleaningAbortedException => cleaned.delete() - throw e + throw e } } @@ -380,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) { @@ -394,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) @@ -422,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 */ @@ -471,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 @@ -500,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 @@ -529,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() @@ -549,7 +615,8 @@ 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() @@ -557,21 +624,28 @@ private case class CleanerStats(time: Time = SystemTime) { 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 } @@ -589,6 +663,7 @@ private case class CleanerStats(time: Time = SystemTime) { mapBytesRead = 0L mapMessagesRead = 0L messagesRead = 0L + invalidMessagesRead = 0L messagesWritten = 0L bufferUtilization = 0.0d } @@ -603,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 e8ced6a592250..f6795d3ffdb85 --- a/core/src/main/scala/kafka/log/LogCleanerManager.scala +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala @@ -24,7 +24,7 @@ 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} @@ -44,9 +44,12 @@ private[log] case object LogCleaningPaused extends LogCleaningState 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]() @@ -75,13 +78,31 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To def grabFilthiestLog(): Option[LogToClean] = { inLock(lock) { val lastClean = allCleanerCheckpoints() - val dirtyLogs = logs.filter(l => l._2.config.compact) // 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, // create a LogToClean instance for each - lastClean.getOrElse(l._1, l._2.logSegments.head.baseOffset))) - .filter(l => l.totalBytes > 0) // skip any empty logs + 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 - val cleanableLogs = dirtyLogs.filter(l => l.cleanableRatio > l.log.config.minCleanableRatio) // and must meet the minimum threshold for dirty byte ratio + // 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 { @@ -101,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)) } /** @@ -131,8 +152,8 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To } 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)) } /** @@ -160,14 +181,14 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To /** * 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 } } @@ -181,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. */ @@ -188,9 +217,7 @@ 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) 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 5746ad4767589..7fc7e33bc770d --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -18,88 +18,79 @@ 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 object Defaults { - val SegmentSize = 1024 * 1024 - val SegmentMs = Long.MaxValue - val FlushInterval = Long.MaxValue - val FlushMs = Long.MaxValue - val RetentionSize = Long.MaxValue - val RetentionMs = Long.MaxValue - val MaxMessageSize = Int.MaxValue - val MaxIndexSize = 1024 * 1024 - val IndexInterval = 4096 - val FileDeleteDelayMs = 60 * 1000L - val DeleteRetentionMs = 24 * 60 * 60 * 1000L - val MinCleanableDirtyRatio = 0.5 - val Compact = false - val UncleanLeaderElectionEnable = true + 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 } -/** - * 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 compact Should old segments in this log be deleted or deduplicated? - * @param uncleanLeaderElectionEnable Indicates whether unclean leader election is enabled; actually a controller-level property - * but included here for topic-specific configuration validation purposes - */ -case class LogConfig(val segmentSize: Int = Defaults.SegmentSize, - val segmentMs: Long = Defaults.SegmentMs, - val flushInterval: Long = Defaults.FlushInterval, - val flushMs: Long = Defaults.FlushMs, - val retentionSize: Long = Defaults.RetentionSize, - val retentionMs: Long = Defaults.RetentionMs, - val maxMessageSize: Int = Defaults.MaxMessageSize, - val maxIndexSize: Int = Defaults.MaxIndexSize, - val indexInterval: Int = Defaults.IndexInterval, - val fileDeleteDelayMs: Long = Defaults.FileDeleteDelayMs, - val deleteRetentionMs: Long = Defaults.DeleteRetentionMs, - val minCleanableRatio: Double = Defaults.MinCleanableDirtyRatio, - val compact: Boolean = Defaults.Compact, - val uncleanLeaderElectionEnable: Boolean = Defaults.UncleanLeaderElectionEnable) { - - 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(compact) "compact" else "delete") - props.put(UncleanLeaderElectionEnableProp, uncleanLeaderElectionEnable.toString) - props - } - +case class LogConfig(props: java.util.Map[_, _]) extends AbstractConfig(LogConfig.configDef, props, false) { + /** + * Important note: Any configuration parameter that is passed along from KafkaConfig to LogConfig + * should also go in copyKafkaConfigToLog. + */ + 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" @@ -107,72 +98,102 @@ object LogConfig { val MinCleanableDirtyRatioProp = "min.cleanable.dirty.ratio" val CleanupPolicyProp = "cleanup.policy" val UncleanLeaderElectionEnableProp = "unclean.leader.election.enable" - - val ConfigNames = Set(SegmentBytesProp, - SegmentMsProp, - SegmentIndexBytesProp, - FlushMessagesProp, - FlushMsProp, - RetentionBytesProp, - RententionMsProp, - MaxMessageBytesProp, - IndexIntervalBytesProp, - FileDeleteDelayMsProp, - DeleteRetentionMsProp, - MinCleanableDirtyRatioProp, - CleanupPolicyProp, - UncleanLeaderElectionEnableProp) - - - /** - * Parse the given properties instance into a LogConfig object - */ - def fromProps(props: Properties): LogConfig = { - new LogConfig(segmentSize = props.getProperty(SegmentBytesProp, Defaults.SegmentSize.toString).toInt, - segmentMs = props.getProperty(SegmentMsProp, Defaults.SegmentMs.toString).toLong, - maxIndexSize = props.getProperty(SegmentIndexBytesProp, Defaults.MaxIndexSize.toString).toInt, - flushInterval = props.getProperty(FlushMessagesProp, Defaults.FlushInterval.toString).toLong, - flushMs = props.getProperty(FlushMsProp, Defaults.FlushMs.toString).toLong, - retentionSize = props.getProperty(RetentionBytesProp, Defaults.RetentionSize.toString).toLong, - retentionMs = props.getProperty(RententionMsProp, Defaults.RetentionMs.toString).toLong, - maxMessageSize = props.getProperty(MaxMessageBytesProp, Defaults.MaxMessageSize.toString).toInt, - indexInterval = props.getProperty(IndexIntervalBytesProp, Defaults.IndexInterval.toString).toInt, - fileDeleteDelayMs = props.getProperty(FileDeleteDelayMsProp, Defaults.FileDeleteDelayMs.toString).toInt, - deleteRetentionMs = props.getProperty(DeleteRetentionMsProp, Defaults.DeleteRetentionMs.toString).toLong, - minCleanableRatio = props.getProperty(MinCleanableDirtyRatioProp, - Defaults.MinCleanableDirtyRatio.toString).toDouble, - compact = props.getProperty(CleanupPolicyProp, if(Defaults.Compact) "compact" else "delete") - .trim.toLowerCase != "delete", - uncleanLeaderElectionEnable = props.getProperty(UncleanLeaderElectionEnableProp, - Defaults.UncleanLeaderElectionEnable.toString).toBoolean) + 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 4d2924d04bc4b..69386c17153e5 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -57,8 +57,9 @@ class LogManager(val logDirs: Array[File], private val dirLocks = lockLogDirs(logDirs) private val recoveryPointCheckpoints = logDirs.map(dir => (dir, new OffsetCheckpoint(new File(dir, RecoveryPointCheckpointFile)))).toMap loadLogs() - - private val cleaner: LogCleaner = + + // public, so we can access this from kafka.admin.DeleteTopicTest + val cleaner: LogCleaner = if(cleanerConfig.enableCleaner) new LogCleaner(cleanerConfig, logDirs, logs, time = time) else @@ -125,16 +126,24 @@ class LogManager(val logDirs: Array[File], brokerState.newState(RecoveringFromUncleanShutdown) } - val recoveryPoints = this.recoveryPointCheckpoints(dir).read + 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 { - Utils.runnable { + CoreUtils.runnable { debug("Loading log '" + logDir.getName + "'") - val topicPartition = Log.parseTopicPartitionName(logDir.getName) + val topicPartition = Log.parseTopicPartitionName(logDir) val config = topicConfigs.getOrElse(topicPartition.topic, defaultConfig) val logRecoveryPoint = recoveryPoints.getOrElse(topicPartition, 0L) @@ -209,7 +218,7 @@ class LogManager(val logDirs: Array[File], // stop the cleaner first if (cleaner != null) { - Utils.swallow(cleaner.shutdown()) + CoreUtils.swallow(cleaner.shutdown()) } // close logs in each dir @@ -222,7 +231,7 @@ class LogManager(val logDirs: Array[File], val logsInDir = logsByDir.getOrElse(dir.toString, Map()).values val jobsForDir = logsInDir map { log => - Utils.runnable { + CoreUtils.runnable { // flush the log to ensure latest possible recovery point log.flush() log.close() @@ -243,7 +252,7 @@ class LogManager(val logDirs: Array[File], // mark that the shutdown was clean by creating marker file debug("Writing clean shutdown marker at " + dir) - Utils.swallow(new File(dir, Log.CleanShutdownFile).createNewFile()) + CoreUtils.swallow(new File(dir, Log.CleanShutdownFile).createNewFile()) } } catch { case e: ExecutionException => { @@ -355,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 } } @@ -370,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, @@ -404,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) } 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 7597d309f37a0..4de4c2b5d004c --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -44,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 */ @@ -110,12 +112,13 @@ class LogSegment(val log: FileMessageSet, * @param startOffset A lower bound on the first offset to include in the message set we read * @param maxSize The maximum number of bytes to include in the message set we read * @param maxOffset An optional maximum offset for the message set we read + * @param maxPosition An optional maximum position in the log segment that should be exposed for read. * * @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): FetchDataInfo = { + def read(startOffset: Long, maxOffset: Option[Long], maxSize: Int, maxPosition: Long = size): FetchDataInfo = { if(maxSize < 0) throw new IllegalArgumentException("Invalid max size for log read (%d)".format(maxSize)) @@ -136,8 +139,8 @@ class LogSegment(val log: FileMessageSet, val length = maxOffset match { case None => - // no max offset, just use the max size they gave unmolested - maxSize + // no max offset, just read until the max position + min((maxPosition - startPosition.position).toInt, maxSize) case Some(offset) => { // there is a max offset, translate it to a file position and use that to calculate the max read size if(offset < startOffset) @@ -148,7 +151,7 @@ class LogSegment(val log: FileMessageSet, logSize // the max offset is off the end of the log, use the end of the file else mapping.position - min(endPosition - startPosition.position, maxSize) + min(min(maxPosition, endPosition) - startPosition.position, maxSize).toInt } } FetchDataInfo(offsetMetadata, log.read(startPosition.position, length)) @@ -180,7 +183,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 @@ -252,10 +255,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)) } @@ -264,8 +267,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 1c4c7bd89e19e..84d18bd4a1ffc --- 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()) } } @@ -114,7 +114,7 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi /** * Find the largest offset less than or equal to the given targetOffset - * and return a pair holding this offset and it's corresponding physical file position. + * and return a pair holding this offset and its corresponding physical file position. * * @param targetOffset The offset to look up. * @@ -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() } @@ -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 788c7864bc881..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,15 +17,16 @@ package kafka.message -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 @@ -36,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) + } } /** @@ -92,7 +119,7 @@ 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(val buffer: ByteBuffer) extends MessageSet with Logging { private var shallowValidByteCount = -1 @@ -100,7 +127,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi 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:_*)) } @@ -123,7 +150,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi } 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. @@ -147,7 +174,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi 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 @@ -156,12 +183,12 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi 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) @@ -176,7 +203,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi innerIter = null new MessageAndOffset(newMessage, offset) case _ => - innerIter = ByteBufferMessageSet.decompress(newMessage).internalIterator() + innerIter = ByteBufferMessageSet.deepIterator(newMessage) if(!innerIter.hasNext) innerIter = null makeNext() @@ -191,42 +218,62 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi 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 */ @@ -237,7 +284,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi */ 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 de0a0fade5387..4d7ce1774f0e5 100644 --- a/core/src/main/scala/kafka/message/CompressionCodec.scala +++ b/core/src/main/scala/kafka/message/CompressionCodec.scala @@ -24,7 +24,6 @@ object CompressionCodec { case GZIPCompressionCodec.codec => GZIPCompressionCodec case SnappyCompressionCodec.codec => SnappyCompressionCodec case LZ4CompressionCodec.codec => LZ4CompressionCodec - case LZ4HCCompressionCodec.codec => LZ4HCCompressionCodec case _ => throw new kafka.common.UnknownCodecException("%d is an unknown compression codec".format(codec)) } } @@ -34,40 +33,65 @@ object CompressionCodec { case GZIPCompressionCodec.name => GZIPCompressionCodec case SnappyCompressionCodec.name => SnappyCompressionCodec case LZ4CompressionCodec.name => LZ4CompressionCodec - case LZ4HCCompressionCodec.name => LZ4HCCompressionCodec 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 LZ4CompressionCodec extends CompressionCodec { +case object LZ4CompressionCodec extends CompressionCodec with BrokerCompressionCodec { val codec = 3 val name = "lz4" } -case object LZ4HCCompressionCodec extends CompressionCodec { - val codec = 4 - val name = "lz4hc" -} - -case object NoCompressionCodec extends CompressionCodec { +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 8420e13d0d868..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 = { @@ -32,11 +34,7 @@ object CompressionFactory { import org.xerial.snappy.SnappyOutputStream new SnappyOutputStream(stream) case LZ4CompressionCodec => - import net.jpountz.lz4.LZ4BlockOutputStream - new LZ4BlockOutputStream(stream) - case LZ4HCCompressionCodec => - import net.jpountz.lz4.{LZ4BlockOutputStream, LZ4Factory} - new LZ4BlockOutputStream(stream, 1 << 16, LZ4Factory.fastestInstance().highCompressor()) + new KafkaLZ4BlockOutputStream(stream) case _ => throw new kafka.common.UnknownCodecException("Unknown Codec: " + compressionCodec) } @@ -49,9 +47,8 @@ object CompressionFactory { case SnappyCompressionCodec => import org.xerial.snappy.SnappyInputStream new SnappyInputStream(stream) - case LZ4CompressionCodec | LZ4HCCompressionCodec => - import net.jpountz.lz4.LZ4BlockInputStream - new LZ4BlockInputStream(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 d2a7293c7be40..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 @@ -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 f1b8432f4a96f..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 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 2313a57d02c06..72ecae1f3484f 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, @@ -35,29 +35,58 @@ 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=") + + nameBuilder.append(typeName) + + 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 newHistogram(name: String, biased: Boolean = true) = - Metrics.defaultRegistry().newHistogram(metricName(name), biased) + def newGauge[T](name: String, metric: Gauge[T], tags: scala.collection.Map[String, String] = Map.empty) = + Metrics.defaultRegistry().newGauge(metricName(name, tags), metric) - def newTimer(name: String, durationUnit: TimeUnit, rateUnit: TimeUnit) = - Metrics.defaultRegistry().newTimer(metricName(name), durationUnit, rateUnit) + 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)) - def removeMetric(name: String) = - Metrics.defaultRegistry().removeMetric(metricName(name)) } @@ -68,72 +97,94 @@ object KafkaMetricsGroup extends KafkaMetricsGroup with Logging { */ 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"), - new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "AllTopicsOwnedPartitionsCount"), + 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"), + new MetricName("kafka.consumer", "ConsumerFetcherManager", "MaxLag"), + new MetricName("kafka.consumer", "ConsumerFetcherManager", "MinFetchRate"), // kafka.server.AbstractFetcherThread <-- kafka.consumer.ConsumerFetcherThread - new MetricName("kafka.server", "FetcherLagMetrics", "-ConsumerLag"), + new MetricName("kafka.server", "FetcherLagMetrics", "ConsumerLag"), // kafka.consumer.ConsumerTopicStats <-- kafka.consumer.{ConsumerIterator, PartitionTopicInfo} - new MetricName("kafka.consumer", "ConsumerTopicMetrics", "-MessagesPerSec"), - new MetricName("kafka.consumer", "ConsumerTopicMetrics", "-AllTopicsMessagesPerSec"), + new MetricName("kafka.consumer", "ConsumerTopicMetrics", "MessagesPerSec"), // kafka.consumer.ConsumerTopicStats - new MetricName("kafka.consumer", "ConsumerTopicMetrics", "-BytesPerSec"), - new MetricName("kafka.consumer", "ConsumerTopicMetrics", "-AllTopicsBytesPerSec"), + new MetricName("kafka.consumer", "ConsumerTopicMetrics", "BytesPerSec"), // kafka.server.AbstractFetcherThread <-- kafka.consumer.ConsumerFetcherThread - new MetricName("kafka.server", "FetcherStats", "-BytesPerSec"), - new MetricName("kafka.server", "FetcherStats", "-RequestsPerSec"), + 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"), - new MetricName("kafka.consumer", "FetchRequestAndResponseMetrics", "-AllBrokersFetchResponseSize"), - new MetricName("kafka.consumer", "FetchRequestAndResponseMetrics", "-AllBrokersFetchRequestRateAndTimeMs"), + new MetricName("kafka.consumer", "FetchRequestAndResponseMetrics", "FetchResponseSize"), + new MetricName("kafka.consumer", "FetchRequestAndResponseMetrics", "FetchRequestRateAndTimeMs"), + new MetricName("kafka.consumer", "FetchRequestAndResponseMetrics", "FetchRequestThrottleRateAndTimeMs"), /** * 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"), - new MetricName("kafka.producer", "ProducerRequestMetrics", "-AllBrokersProducerRequestRateAndTimeMs"), - new MetricName("kafka.producer", "ProducerRequestMetrics", "-AllBrokersProducerRequestSize") + new MetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestRateAndTimeMs"), + new MetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestSize") ) - private val producerMetricNameList: immutable.List[MetricName] = immutable.List[MetricName] ( + 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"), + 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"), + 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"), - new MetricName("kafka.producer", "ProducerTopicMetrics", "-AllTopicsMessagesPerSec"), - new MetricName("kafka.producer", "ProducerTopicMetrics", "-AllTopicsDroppedMessagesPerSec"), - new MetricName("kafka.producer", "ProducerTopicMetrics", "-AllTopicsBytesPerSec"), + 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"), - new MetricName("kafka.producer", "ProducerRequestMetrics", "-AllBrokersProducerRequestRateAndTimeMs"), - new MetricName("kafka.producer", "ProducerRequestMetrics", "-AllBrokersProducerRequestSize") + new MetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestRateAndTimeMs"), + new MetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestSize"), + new MetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestThrottleRateAndTimeMs") ) + 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) @@ -150,18 +201,19 @@ object KafkaMetricsGroup extends KafkaMetricsGroup with Logging { private def removeAllMetricsInList(metricNameList: immutable.List[MetricName], clientId: String) { metricNameList.foreach(metric => { - val pattern = (clientId + ".*" + metric.getName +".*").r + 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.getName) match { + 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)) + 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 eb7bb14d94cb3..5408e0d34c7d9 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,7 +44,9 @@ 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) { try { @@ -55,19 +59,28 @@ class BlockingChannel( val host: String, channel.socket.setSoTimeout(readTimeoutMs) channel.socket.setKeepAlive(true) channel.socket.setTcpNoDelay(true) - channel.connect(new InetSocketAddress(host, port)) + 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)." + 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)) + writeBufferSize, + connectTimeoutMs)) + } catch { case e: Throwable => disconnect() } @@ -92,21 +105,28 @@ class BlockingChannel( val host: String, 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() - response.readCompletely(readChannel) + val response = readCompletely(readChannel) + response.payload().rewind() + + response + } + private def readCompletely(channel: ReadableByteChannel): NetworkReceive = { + val response = new NetworkReceive + while (!response.complete()) + response.readFromReadableChannel(channel) response } 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 4560d8fb7dbfe..798e01d71a286 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -17,20 +17,25 @@ package kafka.network +import java.nio.ByteBuffer +import java.security.Principal 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.kafka.common.security.auth.KafkaPrincipal 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", new Session(KafkaPrincipal.ANONYMOUS, ""), buffer = getShutdownReceive(), startTimeMs = 0, securityProtocol = SecurityProtocol.PLAINTEXT) def getShutdownReceive() = { val emptyProducerRequest = new ProducerRequest(0, 0, "", 0, 0, collection.mutable.Map[TopicAndPartition, ByteBufferMessageSet]()) @@ -41,16 +46,45 @@ 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 Session(principal: KafkaPrincipal, host: String) + + case class Request(processor: Int, connectionId: String, session: Session, 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(details: Boolean): String = { + if (requestObj != null) + requestObj.describe(details) + else + header.toString + " -- " + body.toString + } + + trace("Processor %d received request : %s".format(processor, requestDesc(true))) def updateRequestMetrics() { val endTimeMs = SystemTime.milliseconds @@ -64,7 +98,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 +115,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,securityProtocol:%s,principal:%s" + .format(requestDesc(true), connectionId, totalTime, requestQueueTime, apiLocalTime, apiRemoteTime, responseQueueTime, responseSendTime, securityProtocol, session.principal)) + else if(requestLogger.isDebugEnabled) + requestLogger.debug("Completed request:%s from connection %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d,securityProtocol:%s,principal:%s" + .format(requestDesc(false), connectionId, totalTime, requestQueueTime, apiLocalTime, apiRemoteTime, responseQueueTime, responseSendTime, securityProtocol, session.principal)) } } @@ -125,12 +159,12 @@ class RequestChannel(val numProcessors: Int, val queueSize: Int) extends KafkaMe def value = responseQueues.foldLeft(0) {(total, q) => total + q.size()} }) - for(i <- 0 until numProcessors) { - newGauge( - "Processor-" + i + "-ResponseQueueSize", + for (i <- 0 until numProcessors) { + newGauge("ResponseQueueSize", new Gauge[Int] { def value = responseQueues(i).size() - } + }, + Map("processor" -> i.toString) ) } @@ -187,24 +221,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 cee76b323e5f3..41a3705e62994 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -17,89 +17,143 @@ package kafka.network +import java.io.IOException +import java.net._ +import java.nio.channels._ +import java.nio.channels.{Selector => NSelector} import java.util import java.util.concurrent._ import java.util.concurrent.atomic._ -import java.net._ -import java.io._ -import java.nio.channels._ - -import scala.collection._ +import com.yammer.metrics.core.Gauge +import kafka.cluster.{BrokerEndPoint, EndPoint} import kafka.common.KafkaException import kafka.metrics.KafkaMetricsGroup +import kafka.server.KafkaConfig import kafka.utils._ -import com.yammer.metrics.core.{Gauge, Meter} +import org.apache.kafka.common.MetricName +import org.apache.kafka.common.metrics._ +import org.apache.kafka.common.network.{Selector => KSelector, ChannelBuilders, InvalidReceiveException} +import org.apache.kafka.common.security.auth.KafkaPrincipal +import org.apache.kafka.common.security.ssl.SSLFactory +import org.apache.kafka.common.protocol.SecurityProtocol +import org.apache.kafka.common.protocol.types.SchemaException +import org.apache.kafka.common.utils.{Time, Utils} + +import scala.collection._ +import JavaConverters._ +import scala.util.control.{NonFatal, ControlThrowable} /** * An NIO socket server. The threading model is * 1 Acceptor thread that handles new connections - * N Processor threads that each have their own selector and read requests from sockets + * Acceptor has N Processor threads that each have their own selector and read requests from sockets * 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 numProcessorThreads: Int, - val maxQueuedRequests: Int, - val sendBufferSize: Int, - val recvBufferSize: Int, - val maxRequestSize: Int = Int.MaxValue, - val maxConnectionsPerIp: Int = Int.MaxValue, - val connectionsMaxIdleMs: Long, - val maxConnectionsPerIpOverrides: Map[String, Int] = Map[String, Int]()) 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 - val requestChannel = new RequestChannel(numProcessorThreads, maxQueuedRequests) - - /* a meter to track the average free capacity of the network processors */ - private val aggregateIdleMeter = newMeter("NetworkProcessorAvgIdlePercent", "percent", TimeUnit.NANOSECONDS) +class SocketServer(val config: KafkaConfig, val metrics: Metrics, val time: Time) extends Logging with KafkaMetricsGroup { + + private val endpoints = config.listeners + private val numProcessorThreads = config.numNetworkThreads + private val maxQueuedRequests = config.queuedMaxRequests + private val totalProcessorThreads = numProcessorThreads * endpoints.size + + private val maxConnectionsPerIp = config.maxConnectionsPerIp + private val maxConnectionsPerIpOverrides = config.maxConnectionsPerIpOverrides + + this.logIdent = "[Socket Server on Broker " + config.brokerId + "], " + + val requestChannel = new RequestChannel(totalProcessorThreads, maxQueuedRequests) + private val processors = new Array[Processor](totalProcessorThreads) + + private[network] val acceptors = mutable.Map[EndPoint, Acceptor]() + private var connectionQuotas: ConnectionQuotas = _ + + private val allMetricNames = (0 until totalProcessorThreads).map { i => + val tags = new util.HashMap[String, String]() + tags.put("networkProcessor", i.toString) + new MetricName("io-wait-ratio", "socket-server-metrics", tags) + } /** * Start the socket server */ def startup() { - val quotas = new ConnectionQuotas(maxConnectionsPerIp, maxConnectionsPerIpOverrides) - for(i <- 0 until numProcessorThreads) { - processors(i) = new Processor(i, - time, - maxRequestSize, - aggregateIdleMeter, - newMeter("NetworkProcessor-" + i + "-IdlePercent", "percent", TimeUnit.NANOSECONDS), - numProcessorThreads, - requestChannel, - quotas, - connectionsMaxIdleMs) - Utils.newThread("kafka-network-thread-%d-%d".format(port, i), processors(i), false).start() + this.synchronized { + + connectionQuotas = new ConnectionQuotas(maxConnectionsPerIp, maxConnectionsPerIpOverrides) + + val channelConfigs = config.channelConfigs + val sendBufferSize = config.socketSendBufferBytes + val recvBufferSize = config.socketReceiveBufferBytes + val maxRequestSize = config.socketRequestMaxBytes + val connectionsMaxIdleMs = config.connectionsMaxIdleMs + val brokerId = config.brokerId + + var processorBeginIndex = 0 + endpoints.values.foreach { endpoint => + val protocol = endpoint.protocolType + val processorEndIndex = processorBeginIndex + numProcessorThreads + + for (i <- processorBeginIndex until processorEndIndex) { + processors(i) = new Processor(i, + time, + maxRequestSize, + requestChannel, + connectionQuotas, + connectionsMaxIdleMs, + protocol, + channelConfigs, + metrics + ) + } + + val acceptor = new Acceptor(endpoint, sendBufferSize, recvBufferSize, brokerId, + processors.slice(processorBeginIndex, processorEndIndex), connectionQuotas) + acceptors.put(endpoint, acceptor) + Utils.newThread("kafka-socket-acceptor-%s-%d".format(protocol.toString, endpoint.port), acceptor, false).start() + acceptor.awaitStartup() + + processorBeginIndex = processorEndIndex + } } - newGauge("ResponsesBeingSent", new Gauge[Int] { - def value = processors.foldLeft(0) { (total, p) => total + p.countInterestOps(SelectionKey.OP_WRITE) } - }) - - // 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, quotas) - Utils.newThread("kafka-socket-acceptor", acceptor, false).start() - acceptor.awaitStartup - info("Started") + newGauge("NetworkProcessorAvgIdlePercent", + new Gauge[Double] { + def value = allMetricNames.map( metricName => + metrics.metrics().get(metricName).value()).sum / totalProcessorThreads + } + ) + + info("Started " + acceptors.size + " acceptor threads") } + // register the processor threads for notification of responses + requestChannel.addResponseListener(id => processors(id).wakeup()) + /** * Shutdown the socket server */ 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) + } + } + + /* For test usage */ + private[network] def connectionCount(address: InetAddress): Int = + Option(connectionQuotas).fold(0)(_.get(address)) + } /** @@ -107,18 +161,19 @@ class SocketServer(val brokerId: Int, */ 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(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() } /** @@ -130,121 +185,117 @@ private[kafka] abstract class AbstractServerThread(connectionQuotas: ConnectionQ * Record that the thread startup is complete */ protected def startupComplete() = { - 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? */ protected def isRunning = alive.get - - /** - * Wakeup the thread for selection. - */ - def wakeup() = selector.wakeup() - + /** - * Close the given key and associated socket + * Close the connection identified by `connectionId` and decrement the connection count. */ - def close(key: SelectionKey) { - if(key != null) { - key.attach(null) - close(key.channel.asInstanceOf[SocketChannel]) - swallowError(key.cancel()) + def close(selector: KSelector, connectionId: String) { + val channel = selector.channel(connectionId) + if (channel != null) { + debug(s"Closing selector connection $connectionId") + val address = channel.socketAddress + if (address != null) + connectionQuotas.dec(address) + selector.close(connectionId) } } - + + /** + * Close `channel` and decrement the connection count. + */ def close(channel: SocketChannel) { - if(channel != null) { + if (channel != null) { debug("Closing connection from " + channel.socket.getRemoteSocketAddress()) connectionQuotas.dec(channel.socket.getInetAddress) swallowError(channel.socket().close()) swallowError(channel.close()) } } - - /** - * Close all open connections - */ - def closeAll() { - // removes cancelled keys from selector.keys set - this.selector.selectNow() - val iter = this.selector.keys().iterator() - while (iter.hasNext) { - val key = iter.next() - close(key) - } - } - - def countInterestOps(ops: Int): Int = { - var count = 0 - val it = this.selector.keys().iterator() - while (it.hasNext) { - if ((it.next().interestOps() & ops) != 0) { - count += 1 - } - } - count - } } /** - * Thread that accepts and configures new connections. There is only need for one of these + * Thread that accepts and configures new connections. There is one of these per endpoint. */ -private[kafka] class Acceptor(val host: String, - val port: Int, - private val processors: Array[Processor], - val sendBufferSize: Int, +private[kafka] class Acceptor(val endPoint: EndPoint, + val sendBufferSize: Int, val recvBufferSize: Int, - connectionQuotas: ConnectionQuotas) extends AbstractServerThread(connectionQuotas) { - val serverChannel = openServerSocket(host, port) + brokerId: Int, + processors: Array[Processor], + connectionQuotas: ConnectionQuotas) extends AbstractServerThread(connectionQuotas) with KafkaMetricsGroup { + + private val nioSelector = NSelector.open() + val serverChannel = openServerSocket(endPoint.host, endPoint.port) + + this.synchronized { + processors.foreach { processor => + Utils.newThread("kafka-network-thread-%d-%s-%d".format(brokerId, endPoint.protocolType.toString, processor.id), processor, false).start() + } + } /** * 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) - 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.isAcceptable) - 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 - } catch { - case e: Throwable => error("Error while accepting connection", e) + try { + var currentProcessor = 0 + while (isRunning) { + try { + val ready = nioSelector.select(500) + if (ready > 0) { + val keys = nioSelector.selectedKeys() + val iter = keys.iterator() + while (iter.hasNext && isRunning) { + try { + val key = iter.next + iter.remove() + if (key.isAcceptable) + 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 + } catch { + case e: Throwable => error("Error while accepting connection", e) + } + } } } + catch { + // We catch all the throwables to prevent the acceptor thread from exiting on exceptions due + // to a select operation on a specific channel or a bad request. We don't want the + // the broker to stop responding to requests from other clients in these scenarios. + case e: ControlThrowable => throw e + case e: Throwable => error("Error occurred", e) + } } + } finally { + debug("Closing server socket and selector.") + swallowError(serverChannel.close()) + swallowError(nioSelector.close()) + shutdownComplete() } - debug("Closing server socket and selector.") - swallowError(serverChannel.close()) - swallowError(selector.close()) - shutdownComplete() } - + /* * Create a server socket to listen for connections on. */ - def openServerSocket(host: String, port: Int): ServerSocketChannel = { - val socketAddress = + private def openServerSocket(host: String, port: Int): ServerSocketChannel = { + val socketAddress = if(host == null || host.trim.isEmpty) new InetSocketAddress(port) else @@ -254,9 +305,9 @@ private[kafka] class Acceptor(val host: String, 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 => + case e: SocketException => throw new KafkaException("Socket server failed to bind to %s:%d: %s.".format(socketAddress.getHostName, port, e.getMessage), e) } serverChannel @@ -272,6 +323,7 @@ private[kafka] class Acceptor(val host: String, 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]" @@ -287,123 +339,156 @@ private[kafka] class Acceptor(val host: String, } } + /** + * Wakeup the thread for selection. + */ + @Override + def wakeup = nioSelector.wakeup() + } /** * Thread that processes all requests from a single connection. There are N of these running in parallel - * each of which has its own selectors + * each of which has its own selector */ private[kafka] class Processor(val id: Int, - val time: Time, - val maxRequestSize: Int, - val aggregateIdleMeter: Meter, - val idleMeter: Meter, - val totalProcessorThreads: Int, - val requestChannel: RequestChannel, + time: Time, + maxRequestSize: Int, + requestChannel: RequestChannel, connectionQuotas: ConnectionQuotas, - val connectionsMaxIdleMs: Long) extends AbstractServerThread(connectionQuotas) { + connectionsMaxIdleMs: Long, + protocol: SecurityProtocol, + channelConfigs: java.util.Map[String, Object], + metrics: Metrics) extends AbstractServerThread(connectionQuotas) with KafkaMetricsGroup { + + private object ConnectionId { + def fromString(s: String): Option[ConnectionId] = s.split("-") match { + case Array(local, remote) => BrokerEndPoint.parseHostPort(local).flatMap { case (localHost, localPort) => + BrokerEndPoint.parseHostPort(remote).map { case (remoteHost, remotePort) => + ConnectionId(localHost, localPort, remoteHost, remotePort) + } + } + case _ => None + } + } + + private case class ConnectionId(localHost: String, localPort: Int, remoteHost: String, remotePort: Int) { + override def toString: String = s"$localHost:$localPort-$remoteHost:$remotePort" + } private val newConnections = new ConcurrentLinkedQueue[SocketChannel]() - private val connectionsMaxIdleNanos = connectionsMaxIdleMs * 1000 * 1000 - private var currentTimeNanos = SystemTime.nanoseconds - private val lruConnections = new util.LinkedHashMap[SelectionKey, Long] - private var nextIdleCloseCheckTime = currentTimeNanos + connectionsMaxIdleNanos + private val inflightResponses = mutable.Map[String, RequestChannel.Response]() + private val channelBuilder = ChannelBuilders.create(protocol, SSLFactory.Mode.SERVER, channelConfigs) + 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() + } + }, + metricTags.asScala + ) + + private val selector = new KSelector( + maxRequestSize, + connectionsMaxIdleMs, + metrics, + time, + "socket-server", + metricTags, + false, + channelBuilder) override def run() { startupComplete() while(isRunning) { - // setup any new connections that have been queued up - configureNewConnections() - // register any new responses for writing - processNewResponses() - val startSelectTime = SystemTime.nanoseconds - val ready = selector.select(300) - currentTimeNanos = SystemTime.nanoseconds - val idleTime = currentTimeNanos - startSelectTime - idleMeter.mark(idleTime) - // 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. - aggregateIdleMeter.mark(idleTime / totalProcessorThreads) - - trace("Processor id " + id + " selection time = " + idleTime + " ns") - if(ready > 0) { - val keys = selector.selectedKeys() - val iter = keys.iterator() - while(iter.hasNext && isRunning) { - var key: SelectionKey = null + try { + // setup any new connections that have been queued up + configureNewConnections() + // register any new responses for writing + processNewResponses() + + 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") + } + selector.completedReceives.asScala.foreach { receive => 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.") + val channel = selector.channel(receive.source) + val session = RequestChannel.Session(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, channel.principal().getName), channel.socketDescription) + val req = RequestChannel.Request(processor = id, connectionId = receive.source, session = session, buffer = receive.payload, startTimeMs = time.milliseconds, securityProtocol = protocol) + requestChannel.sendRequest(req) } 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) - } + 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) + close(selector, receive.source) } + selector.mute(receive.source) + } + + selector.completedSends.asScala.foreach { send => + val resp = inflightResponses.remove(send.destination()).getOrElse { + throw new IllegalStateException(s"Send for ${send.destination} completed, but not in `inflightResponses`") + } + resp.request.updateRequestMetrics() + selector.unmute(send.destination()) + } + + selector.disconnected.asScala.foreach { connectionId => + val remoteHost = ConnectionId.fromString(connectionId).getOrElse { + throw new IllegalStateException(s"connectionId has unexpected format: $connectionId") + }.remoteHost + // the channel has been closed by the selector but the quotas still need to be updated + connectionQuotas.dec(InetAddress.getByName(remoteHost)) } + + } catch { + // We catch all the throwables here to prevent the processor thread from exiting. We do this because + // letting a processor exit might cause bigger impact on the broker. Usually the exceptions thrown would + // be either associated with a specific socket channel or a bad request. We just ignore the bad socket channel + // or request. This behavior might need to be reviewed if we see an exception that need the entire broker to stop. + case e : ControlThrowable => throw e + case e : Throwable => + error("Processor got uncaught exception.", e) } - maybeCloseOldestConnection } - debug("Closing selector.") - closeAll() - swallowError(selector.close()) - shutdownComplete() - } - /** - * Close the given key and associated socket - */ - override def close(key: SelectionKey): Unit = { - lruConnections.remove(key) - super.close(key) + debug("Closing selector - processor " + id) + swallowError(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 => { + case RequestChannel.NoOpAction => // There is no response to send to the client, we need to read more pipelined requests // that are sitting in the server's socket buffer curr.request.updateRequestMetrics trace("Socket server received empty response to send, registering for read: " + curr) - key.interestOps(SelectionKey.OP_READ) - key.attach(null) - } - case RequestChannel.SendAction => { - trace("Socket server received response to send, registering for write: " + curr) - key.interestOps(SelectionKey.OP_WRITE) - key.attach(curr) - } - case RequestChannel.CloseConnectionAction => { + selector.unmute(curr.request.connectionId) + case RequestChannel.SendAction => + 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) - } - case responseCode => throw new KafkaException("No mapping found for response code " + responseCode) - } - } catch { - case e: CancelledKeyException => { - debug("Ignoring response for closed socket.") - close(key) + close(selector, curr.request.connectionId) } } finally { curr = requestChannel.receiveResponse(id) @@ -423,112 +508,76 @@ 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) + try { + debug("Processor " + id + " listening to new connection from " + channel.socket.getRemoteSocketAddress) + val localHost = channel.socket().getLocalAddress.getHostAddress + val localPort = channel.socket().getLocalPort + val remoteHost = channel.socket().getInetAddress.getHostAddress + val remotePort = channel.socket().getPort + val connectionId = ConnectionId(localHost, localPort, remoteHost, remotePort).toString + selector.register(connectionId, channel) + } catch { + // We explicitly catch all non fatal exceptions and close the socket to avoid socket leak. The other + // throwables will be caught in processor and logged as uncaught exception. + case NonFatal(e) => + // need to close the channel here to avoid socket leak. + close(channel) + error("Processor " + id + " closed connection from " + channel.getRemoteAddress, e) + } } } - /* - * Process reads from ready sockets + /** + * Close the selector and all open connections */ - def read(key: SelectionKey) { - lruConnections.put(key, currentTimeNanos) - 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() + private def closeAll() { + selector.channels.asScala.foreach { channel => + close(selector, channel.id) } + selector.close() } - /* - * Process writes to ready sockets - */ - 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() - } - } - private def channelFor(key: SelectionKey) = key.channel().asInstanceOf[SocketChannel] - - private def maybeCloseOldestConnection { - if(currentTimeNanos > nextIdleCloseCheckTime) { - if(lruConnections.isEmpty) { - nextIdleCloseCheckTime = currentTimeNanos + connectionsMaxIdleNanos - } else { - val oldestConnectionEntry = lruConnections.entrySet.iterator().next() - val connectionLastActiveTime = oldestConnectionEntry.getValue - nextIdleCloseCheckTime = connectionLastActiveTime + connectionsMaxIdleNanos - if(currentTimeNanos > nextIdleCloseCheckTime) { - val key: SelectionKey = oldestConnectionEntry.getKey - trace("About to close the idle connection from " + key.channel.asInstanceOf[SocketChannel].socket.getRemoteSocketAddress - + " due to being idle for " + (currentTimeNanos - connectionLastActiveTime) / 1000 / 1000 + " millis") - close(key) - } - } - } - } + /** + * Wakeup the thread for selection. + */ + @Override + def wakeup = selector.wakeup() } class ConnectionQuotas(val defaultMax: Int, overrideQuotas: Map[String, Int]) { - private val overrides = overrideQuotas.map(entry => (InetAddress.getByName(entry._1), entry._2)) + + private val overrides = overrideQuotas.map { case (host, count) => (InetAddress.getByName(host), count) } 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 inc(address: InetAddress) { + counts.synchronized { + val count = counts.getOrElseUpdate(address, 0) + counts.put(address, count + 1) + val max = overrides.getOrElse(address, defaultMax) + if (count >= max) + throw new TooManyConnectionsException(address, max) } } - - def dec(addr: InetAddress) { - counts synchronized { - val count = counts.get(addr).get - if(count == 1) - counts.remove(addr) + + def dec(address: InetAddress) { + counts.synchronized { + val count = counts.getOrElse(address, + throw new IllegalArgumentException(s"Attempted to decrease connection count for address with no connections, address: $address")) + if (count == 1) + counts.remove(address) else - counts.put(addr, count - 1) + counts.put(address, count - 1) } } - + + def get(address: InetAddress): Int = counts.synchronized { + counts.getOrElse(address, 0) + } + } 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 index b0207930dd054..8e007130b2825 100644 --- a/core/src/main/scala/kafka/producer/BaseProducer.scala +++ b/core/src/main/scala/kafka/producer/BaseProducer.scala @@ -33,10 +33,10 @@ class NewShinyProducer(producerProps: Properties) extends BaseProducer { // decide whether to send synchronously based on producer properties val sync = producerProps.getProperty("producer.type", "async").equals("sync") - val producer = new KafkaProducer(producerProps) + val producer = new KafkaProducer[Array[Byte],Array[Byte]](producerProps) override def send(topic: String, key: Array[Byte], value: Array[Byte]) { - val record = new ProducerRecord(topic, key, value) + val record = new ProducerRecord[Array[Byte],Array[Byte]](topic, key, value) if(sync) { this.producer.send(record).get() } else { 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 6a3b02e414eb7..e6b100eb0f5de --- a/core/src/main/scala/kafka/producer/ByteArrayPartitioner.scala +++ b/core/src/main/scala/kafka/producer/ByteArrayPartitioner.scala @@ -19,6 +19,7 @@ package kafka.producer import kafka.utils._ +import org.apache.kafka.common.utils.Utils class ByteArrayPartitioner(props: VerifiableProperties = null) extends Partitioner { def partition(key: Any, numPartitions: Int): Int = { 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 4b5b823b85477..0000000000000 --- a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.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.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} -import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} - -class KafkaLog4jAppender extends AppenderSkeleton with Logging { - var topic: String = null - var brokerList: String = null - var compressionType: String = null - var requiredNumAcks: Int = Int.MaxValue - var syncSend: Boolean = false - - private var producer: KafkaProducer = null - - def getTopic: String = topic - def setTopic(topic: String) { this.topic = topic } - - def getBrokerList: String = brokerList - def setBrokerList(brokerList: String) { this.brokerList = brokerList } - - def getCompressionType: String = compressionType - def setCompressionType(compressionType: String) { this.compressionType = compressionType } - - def getRequiredNumAcks: Int = requiredNumAcks - def setRequiredNumAcks(requiredNumAcks: Int) { this.requiredNumAcks = requiredNumAcks } - - def getSyncSend: Boolean = syncSend - def setSyncSend(syncSend: Boolean) { this.syncSend = syncSend } - - override def activateOptions() { - // check for config parameter validity - val props = new Properties() - if(brokerList != null) - props.put(org.apache.kafka.clients.producer.ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) - if(props.isEmpty) - throw new MissingConfigException("The bootstrap servers property should be specified") - if(topic == null) - throw new MissingConfigException("topic must be specified by the Kafka log4j appender") - if(compressionType != null) props.put(org.apache.kafka.clients.producer.ProducerConfig.COMPRESSION_TYPE_CONFIG, compressionType) - if(requiredNumAcks != Int.MaxValue) props.put(org.apache.kafka.clients.producer.ProducerConfig.ACKS_CONFIG, requiredNumAcks.toString) - producer = new KafkaProducer(props) - LogLog.debug("Kafka producer connected to " + 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 response = producer.send(new ProducerRecord(topic, message.getBytes())) - if (syncSend) response.get - } - - 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 = true -} 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 cd634f653caa9..4be06c8d3fc7f --- a/core/src/main/scala/kafka/producer/Producer.scala +++ b/core/src/main/scala/kafka/producer/Producer.scala @@ -19,7 +19,7 @@ package kafka.producer import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.{LinkedBlockingQueue, TimeUnit} -import kafka.common.QueueFullException +import kafka.common.{AppInfo, QueueFullException} import kafka.metrics._ import kafka.producer.async.{DefaultEventHandler, EventHandler, ProducerSendThread} import kafka.serializer.Encoder @@ -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))) /** 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 3cdf23dce3407..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 { @@ -90,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 1c46d729d8259..b453f639b9082 100644 --- a/core/src/main/scala/kafka/producer/ProducerRequestStats.scala +++ b/core/src/main/scala/kafka/producer/ProducerRequestStats.scala @@ -19,11 +19,17 @@ 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) + val throttleTimeStats = newTimer("ProducerRequestThrottleRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS, tags); } /** @@ -31,14 +37,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)) } } diff --git a/core/src/main/scala/kafka/producer/ProducerStats.scala b/core/src/main/scala/kafka/producer/ProducerStats.scala index 35e3aae2f81e5..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) } /** diff --git a/core/src/main/scala/kafka/producer/ProducerTopicStats.scala b/core/src/main/scala/kafka/producer/ProducerTopicStats.scala index 9bb1419dcc45f..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)) } } diff --git a/core/src/main/scala/kafka/producer/SyncProducer.scala b/core/src/main/scala/kafka/producer/SyncProducer.scala index 42c950375098b..73db2b140872c 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,11 +17,13 @@ package kafka.producer -import kafka.api._ -import kafka.network.{BlockingChannel, BoundedByteBufferSend, Receive} -import kafka.utils._ import java.util.Random +import java.util.concurrent.TimeUnit +import kafka.api._ +import kafka.network.{RequestOrResponseSend, BlockingChannel} +import kafka.utils._ +import org.apache.kafka.common.network.NetworkReceive import org.apache.kafka.common.utils.Utils._ object SyncProducer { @@ -39,10 +41,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) = { /** @@ -51,7 +52,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) { @@ -64,12 +65,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) @@ -93,26 +94,30 @@ 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 { response = doSend(producerRequest, if(producerRequest.requiredAcks == 0) false else true) } } - if(producerRequest.requiredAcks != 0) - ProducerResponse.readFrom(response.buffer) + if(producerRequest.requiredAcks != 0) { + val producerResponse = ProducerResponse.readFrom(response.payload) + producerRequestStats.getProducerRequestStats(config.host, config.port).throttleTimeStats.update(producerResponse.throttleTime, TimeUnit.MILLISECONDS) + producerRequestStats.getProducerRequestAllBrokersStats.throttleTimeStats.update(producerResponse.throttleTime, TimeUnit.MILLISECONDS) + producerResponse + } else null } def send(request: TopicMetadataRequest): TopicMetadataResponse = { val response = doSend(request) - TopicMetadataResponse.readFrom(response.buffer) + TopicMetadataResponse.readFrom(response.payload) } def close() = { @@ -134,7 +139,7 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { case e: Exception => error("Error on disconnect: ", e) } } - + private def connect(): BlockingChannel = { if (!blockingChannel.isConnected && !shutdown) { try { @@ -156,5 +161,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 33470fff21bd4..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,27 +96,28 @@ 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 } } @@ -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/security/auth/Acl.scala b/core/src/main/scala/kafka/security/auth/Acl.scala new file mode 100644 index 0000000000000..c23dd2d81f52a --- /dev/null +++ b/core/src/main/scala/kafka/security/auth/Acl.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.security.auth + +import kafka.utils.Json +import org.apache.kafka.common.security.auth.KafkaPrincipal + +object Acl { + val WildCardPrincipal: KafkaPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "*") + val WildCardHost: String = "*" + val AllowAllAcl = new Acl(WildCardPrincipal, Allow, WildCardHost, All) + val PrincipalKey = "principal" + val PermissionTypeKey = "permissionType" + val OperationKey = "operation" + val HostsKey = "host" + val VersionKey = "version" + val CurrentVersion = 1 + val AclsKey = "acls" + + /** + * + * @param aclJson + * + *

              + { + "version": 1, + "acls": [ + { + "host":"host1", + "permissionType": "Deny", + "operation": "Read", + "principal": "User:alice" + } + ] + } + *

              + * + * @return + */ + def fromJson(aclJson: String): Set[Acl] = { + if (aclJson == null || aclJson.isEmpty) + return collection.immutable.Set.empty[Acl] + + var acls: collection.mutable.HashSet[Acl] = new collection.mutable.HashSet[Acl]() + Json.parseFull(aclJson) match { + case Some(m) => + val aclMap = m.asInstanceOf[Map[String, Any]] + //the acl json version. + require(aclMap(VersionKey) == CurrentVersion) + val aclSet: List[Map[String, Any]] = aclMap(AclsKey).asInstanceOf[List[Map[String, Any]]] + aclSet.foreach(item => { + val principal: KafkaPrincipal = KafkaPrincipal.fromString(item(PrincipalKey).asInstanceOf[String]) + val permissionType: PermissionType = PermissionType.fromString(item(PermissionTypeKey).asInstanceOf[String]) + val operation: Operation = Operation.fromString(item(OperationKey).asInstanceOf[String]) + val host: String = item(HostsKey).asInstanceOf[String] + acls += new Acl(principal, permissionType, host, operation) + }) + case None => + } + acls.toSet + } + + def toJsonCompatibleMap(acls: Set[Acl]): Map[String, Any] = { + Map(Acl.VersionKey -> Acl.CurrentVersion, Acl.AclsKey -> acls.map(acl => acl.toMap).toList) + } +} + +/** + * An instance of this class will represent an acl that can express following statement. + *
              + * Principal P has permissionType PT on Operation O1 from hosts H1.
              + * 
              + * @param principal A value of *:* indicates all users. + * @param permissionType + * @param host A value of * indicates all hosts. + * @param operation A value of ALL indicates all operations. + */ +case class Acl(principal: KafkaPrincipal, permissionType: PermissionType, host: String, operation: Operation) { + + /** + * TODO: Ideally we would have a symmetric toJson method but our current json library can not jsonify/dejsonify complex objects. + * @return Map representation of the Acl. + */ + def toMap(): Map[String, Any] = { + Map(Acl.PrincipalKey -> principal.toString, + Acl.PermissionTypeKey -> permissionType.name, + Acl.OperationKey -> operation.name, + Acl.HostsKey -> host) + } + + override def toString: String = { + "%s has %s permission for operations: %s from hosts: %s".format(principal, permissionType.name, operation, host) + } + +} + diff --git a/core/src/main/scala/kafka/security/auth/Authorizer.scala b/core/src/main/scala/kafka/security/auth/Authorizer.scala new file mode 100644 index 0000000000000..8f1a66006092a --- /dev/null +++ b/core/src/main/scala/kafka/security/auth/Authorizer.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.security.auth + +import kafka.network.RequestChannel.Session +import kafka.server.KafkaConfig +import org.apache.kafka.common.Configurable +import org.apache.kafka.common.security.auth.KafkaPrincipal + +/** + * Top level interface that all plugable authorizer must implement. Kafka server will read "authorizer.class" config + * value at startup time, create an instance of the specified class and call initialize method. + * authorizer.class must be a class that implements this interface. + * If authorizer.class has no value specified no authorization will be performed. + * + * From that point onwards, every client request will first be routed to authorize method and the request will only be + * authorized if the method returns true. + */ +trait Authorizer extends Configurable { + + /** + * @param session The session being authenticated. + * @param operation Type of operation client is trying to perform on resource. + * @param resource Resource the client is trying to access. + * @return + */ + def authorize(session: Session, operation: Operation, resource: Resource): Boolean + + /** + * add the acls to resource, this is an additive operation so existing acls will not be overwritten, instead these new + * acls will be added to existing acls. + * @param acls set of acls to add to existing acls + * @param resource the resource to which these acls should be attached. + */ + def addAcls(acls: Set[Acl], resource: Resource): Unit + + /** + * remove these acls from the resource. + * @param acls set of acls to be removed. + * @param resource resource from which the acls should be removed. + * @return true if some acl got removed, false if no acl was removed. + */ + def removeAcls(acls: Set[Acl], resource: Resource): Boolean + + /** + * remove a resource along with all of its acls from acl store. + * @param resource + * @return + */ + def removeAcls(resource: Resource): Boolean + + /** + * get set of acls for this resource + * @param resource + * @return empty set if no acls are found, otherwise the acls for the resource. + */ + def getAcls(resource: Resource): Set[Acl] + + /** + * get the acls for this principal. + * @param principal + * @return empty Map if no acls exist for this principal, otherwise a map of resource -> acls for the principal. + */ + def getAcls(principal: KafkaPrincipal): Map[Resource, Set[Acl]] + + /** + * gets the map of resource to acls for all resources. + */ + def getAcls(): Map[Resource, Set[Acl]] +} + diff --git a/core/src/main/scala/kafka/security/auth/Operation.scala b/core/src/main/scala/kafka/security/auth/Operation.scala new file mode 100644 index 0000000000000..5d31c62a8d96a --- /dev/null +++ b/core/src/main/scala/kafka/security/auth/Operation.scala @@ -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 kafka.security.auth + +import kafka.common.{BaseEnum, KafkaException} + +/** + * Different operations a client may perform on kafka resources. + */ + +sealed trait Operation extends BaseEnum +case object Read extends Operation { val name = "Read" } +case object Write extends Operation { val name = "Write" } +case object Create extends Operation { val name = "Create" } +case object Delete extends Operation { val name = "Delete" } +case object Alter extends Operation { val name = "Alter" } +case object Describe extends Operation { val name = "Describe" } +case object ClusterAction extends Operation { val name = "ClusterAction" } +case object All extends Operation { val name = "All" } + +object Operation { + def fromString(operation: String): Operation = { + val op = values.find(op => op.name.equalsIgnoreCase(operation)) + op.getOrElse(throw new KafkaException(operation + " not a valid operation name. The valid names are " + values.mkString(","))) + } + + def values: Seq[Operation] = List(Read, Write, Create, Delete, Alter, Describe, ClusterAction, All) +} diff --git a/core/src/main/scala/kafka/security/auth/PermissionType.scala b/core/src/main/scala/kafka/security/auth/PermissionType.scala new file mode 100644 index 0000000000000..fd2a0fe4d7972 --- /dev/null +++ b/core/src/main/scala/kafka/security/auth/PermissionType.scala @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

              + * http://www.apache.org/licenses/LICENSE-2.0 + *

              + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.security.auth + +import kafka.common.{BaseEnum, KafkaException} + +/** + * PermissionType. + */ + + +sealed trait PermissionType extends BaseEnum + +case object Allow extends PermissionType { + val name = "Allow" +} + +case object Deny extends PermissionType { + val name = "Deny" +} + +object PermissionType { + def fromString(permissionType: String): PermissionType = { + val pType = values.find(pType => pType.name.equalsIgnoreCase(permissionType)) + pType.getOrElse(throw new KafkaException(permissionType + " not a valid permissionType name. The valid names are " + values.mkString(","))) + } + + def values: Seq[PermissionType] = List(Allow, Deny) +} + diff --git a/core/src/main/scala/kafka/security/auth/Resource.scala b/core/src/main/scala/kafka/security/auth/Resource.scala new file mode 100644 index 0000000000000..797c77bb6ccc0 --- /dev/null +++ b/core/src/main/scala/kafka/security/auth/Resource.scala @@ -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 kafka.security.auth + +object Resource { + val Separator = ":" + val ClusterResourceName = "kafka-cluster" + val ClusterResource = new Resource(Cluster, Resource.ClusterResourceName) + val WildCardResource = "*" + + def fromString(str: String): Resource = { + str.split(Separator, 2) match { + case Array(resourceType, name, _*) => new Resource(ResourceType.fromString(resourceType), name) + case s => throw new IllegalArgumentException("expected a string in format ResourceType:ResourceName but got " + str) + } + } +} + +/** + * + * @param resourceType type of resource. + * @param name name of the resource, for topic this will be topic name , for group it will be group name. For cluster type + * it will be a constant string kafka-cluster. + */ +case class Resource(val resourceType: ResourceType, val name: String) { + + override def toString: String = { + resourceType.name + Resource.Separator + name + } +} + diff --git a/core/src/main/scala/kafka/security/auth/ResourceType.scala b/core/src/main/scala/kafka/security/auth/ResourceType.scala new file mode 100644 index 0000000000000..3b8312d858e89 --- /dev/null +++ b/core/src/main/scala/kafka/security/auth/ResourceType.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.security.auth + +import kafka.common.{BaseEnum, KafkaException} + +/** + * ResourceTypes. + */ + + +sealed trait ResourceType extends BaseEnum + +case object Cluster extends ResourceType { + val name = "Cluster" +} + +case object Topic extends ResourceType { + val name = "Topic" +} + +case object ConsumerGroup extends ResourceType { + val name = "ConsumerGroup" +} + + +object ResourceType { + + def fromString(resourceType: String): ResourceType = { + val rType = values.find(rType => rType.name.equalsIgnoreCase(resourceType)) + rType.getOrElse(throw new KafkaException(resourceType + " not a valid resourceType name. The valid names are " + values.mkString(","))) + } + + def values: Seq[ResourceType] = List(Cluster, Topic, ConsumerGroup) +} diff --git a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala new file mode 100644 index 0000000000000..2e5ee8d99d0de --- /dev/null +++ b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala @@ -0,0 +1,285 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.security.auth + +import java.util +import java.util.concurrent.locks.ReentrantReadWriteLock +import kafka.common.{NotificationHandler, ZkNodeChangeNotificationListener} +import org.apache.zookeeper.Watcher.Event.KeeperState + + +import kafka.network.RequestChannel.Session +import kafka.server.KafkaConfig +import kafka.utils.CoreUtils.{inReadLock, inWriteLock} +import kafka.utils._ +import org.I0Itec.zkclient.{IZkStateListener, ZkClient} +import org.apache.kafka.common.security.auth.KafkaPrincipal +import scala.collection.JavaConverters._ +import org.apache.log4j.Logger + +object SimpleAclAuthorizer { + //optional override zookeeper cluster configuration where acls will be stored, if not specified acls will be stored in + //same zookeeper where all other kafka broker info is stored. + val ZkUrlProp = "authorizer.zookeeper.url" + val ZkConnectionTimeOutProp = "authorizer.zookeeper.connection.timeout.ms" + val ZkSessionTimeOutProp = "authorizer.zookeeper.session.timeout.ms" + + //List of users that will be treated as super users and will have access to all the resources for all actions from all hosts, defaults to no super users. + val SuperUsersProp = "super.users" + //If set to true when no acls are found for a resource , authorizer allows access to everyone. Defaults to false. + val AllowEveryoneIfNoAclIsFoundProp = "allow.everyone.if.no.acl.found" + + /** + * The root acl storage node. Under this node there will be one child node per resource type (Topic, Cluster, ConsumerGroup). + * under each resourceType there will be a unique child for each resource instance and the data for that child will contain + * list of its acls as a json object. Following gives an example: + * + *

              +   * /kafka-acl/Topic/topic-1 => {"version": 1, "acls": [ { "host":"host1", "permissionType": "Allow","operation": "Read","principal": "User:alice"}]}
              +   * /kafka-acl/Cluster/kafka-cluster => {"version": 1, "acls": [ { "host":"host1", "permissionType": "Allow","operation": "Read","principal": "User:alice"}]}
              +   * /kafka-acl/ConsumerGroup/group-1 => {"version": 1, "acls": [ { "host":"host1", "permissionType": "Allow","operation": "Read","principal": "User:alice"}]}
              +   * 
              + */ + val AclZkPath = "/kafka-acl" + + //notification node which gets updated with the resource name when acl on a resource is changed. + val AclChangedZkPath = "/kafka-acl-changes" + + //prefix of all the change notificiation sequence node. + val AclChangedPrefix = "acl_changes_" +} + +class SimpleAclAuthorizer extends Authorizer with Logging { + private val authorizerLogger = Logger.getLogger("kafka.authorizer.logger") + private var superUsers = Set.empty[KafkaPrincipal] + private var shouldAllowEveryoneIfNoAclIsFound = false + private var zkClient: ZkClient = null + private var aclChangeListener: ZkNodeChangeNotificationListener = null + + private val aclCache = new scala.collection.mutable.HashMap[Resource, Set[Acl]] + private val lock = new ReentrantReadWriteLock() + + /** + * Guaranteed to be called before any authorize call is made. + */ + override def configure(javaConfigs: util.Map[String, _]) { + val configs = javaConfigs.asScala + val props = new java.util.Properties() + configs foreach { case (key, value) => props.put(key, value.toString) } + val kafkaConfig = KafkaConfig.fromProps(props) + + superUsers = configs.get(SimpleAclAuthorizer.SuperUsersProp).collect { + case str: String if str.nonEmpty => str.split(",").map(s => KafkaPrincipal.fromString(s.trim)).toSet + }.getOrElse(Set.empty[KafkaPrincipal]) + + shouldAllowEveryoneIfNoAclIsFound = configs.get(SimpleAclAuthorizer.AllowEveryoneIfNoAclIsFoundProp).map(_.toString.toBoolean).getOrElse(false) + + val zkUrl = configs.getOrElse(SimpleAclAuthorizer.ZkUrlProp, kafkaConfig.zkConnect).toString + val zkConnectionTimeoutMs = configs.getOrElse(SimpleAclAuthorizer.ZkConnectionTimeOutProp, kafkaConfig.zkConnectionTimeoutMs).toString.toInt + val zkSessionTimeOutMs = configs.getOrElse(SimpleAclAuthorizer.ZkSessionTimeOutProp, kafkaConfig.zkSessionTimeoutMs).toString.toInt + + zkClient = ZkUtils.createZkClient(zkUrl, zkConnectionTimeoutMs, zkSessionTimeOutMs) + ZkUtils.makeSurePersistentPathExists(zkClient, SimpleAclAuthorizer.AclZkPath) + + loadCache() + + ZkUtils.makeSurePersistentPathExists(zkClient, SimpleAclAuthorizer.AclChangedZkPath) + aclChangeListener = new ZkNodeChangeNotificationListener(zkClient, SimpleAclAuthorizer.AclChangedZkPath, SimpleAclAuthorizer.AclChangedPrefix, AclChangedNotificaitonHandler) + aclChangeListener.init() + + zkClient.subscribeStateChanges(ZkStateChangeListener) + } + + override def authorize(session: Session, operation: Operation, resource: Resource): Boolean = { + val principal: KafkaPrincipal = session.principal + val host = session.host + val acls = getAcls(resource) ++ getAcls(new Resource(resource.resourceType, Resource.WildCardResource)) + + //check if there is any Deny acl match that would disallow this operation. + val denyMatch = aclMatch(session, operation, resource, principal, host, Deny, acls) + + //if principal is allowed to read or write we allow describe by default, the reverse does not apply to Deny. + val ops = if (Describe == operation) + Set[Operation](operation, Read, Write) + else + Set[Operation](operation) + + //now check if there is any allow acl that will allow this operation. + val allowMatch = ops.exists(operation => aclMatch(session, operation, resource, principal, host, Allow, acls)) + + //we allow an operation if a user is a super user or if no acls are found and user has configured to allow all users + //when no acls are found or if no deny acls are found and at least one allow acls matches. + val authorized = isSuperUser(operation, resource, principal, host) || + isEmptyAclAndAuthorized(operation, resource, principal, host, acls) || + (!denyMatch && allowMatch) + + logAuditMessage(principal, authorized, operation, resource, host) + authorized + } + + def isEmptyAclAndAuthorized(operation: Operation, resource: Resource, principal: KafkaPrincipal, host: String, acls: Set[Acl]): Boolean = { + if (acls.isEmpty) { + authorizerLogger.debug(s"No acl found for resource $resource, authorized = $shouldAllowEveryoneIfNoAclIsFound") + shouldAllowEveryoneIfNoAclIsFound + } else false + } + + def isSuperUser(operation: Operation, resource: Resource, principal: KafkaPrincipal, host: String): Boolean = { + if (superUsers.exists( _ == principal)) { + authorizerLogger.debug(s"principal = $principal is a super user, allowing operation without checking acls.") + true + } else false + } + + private def aclMatch(session: Session, operations: Operation, resource: Resource, principal: KafkaPrincipal, host: String, permissionType: PermissionType, acls: Set[Acl]): Boolean = { + acls.find ( acl => + acl.permissionType == permissionType + && (acl.principal == principal || acl.principal == Acl.WildCardPrincipal) + && (operations == acl.operation || acl.operation == All) + && (acl.host == host || acl.host == Acl.WildCardHost) + ).map { acl: Acl => + authorizerLogger.debug(s"operation = $operations on resource = $resource from host = $host is $permissionType based on acl = $acl") + true + }.getOrElse(false) + } + + override def addAcls(acls: Set[Acl], resource: Resource) { + if (acls != null && acls.nonEmpty) { + val updatedAcls = getAcls(resource) ++ acls + val path = toResourcePath(resource) + + if (ZkUtils.pathExists(zkClient, path)) + ZkUtils.updatePersistentPath(zkClient, path, Json.encode(Acl.toJsonCompatibleMap(updatedAcls))) + else + ZkUtils.createPersistentPath(zkClient, path, Json.encode(Acl.toJsonCompatibleMap(updatedAcls))) + + updateAclChangedFlag(resource) + } + } + + override def removeAcls(aclsTobeRemoved: Set[Acl], resource: Resource): Boolean = { + if (ZkUtils.pathExists(zkClient, toResourcePath(resource))) { + val existingAcls = getAcls(resource) + val filteredAcls = existingAcls.filter((acl: Acl) => !aclsTobeRemoved.contains(acl)) + + val aclNeedsRemoval = (existingAcls != filteredAcls) + if (aclNeedsRemoval) { + val path: String = toResourcePath(resource) + if (filteredAcls.nonEmpty) + ZkUtils.updatePersistentPath(zkClient, path, Json.encode(Acl.toJsonCompatibleMap(filteredAcls))) + else + ZkUtils.deletePath(zkClient, toResourcePath(resource)) + + updateAclChangedFlag(resource) + } + + aclNeedsRemoval + } else false + } + + override def removeAcls(resource: Resource): Boolean = { + if (ZkUtils.pathExists(zkClient, toResourcePath(resource))) { + ZkUtils.deletePath(zkClient, toResourcePath(resource)) + updateAclChangedFlag(resource) + true + } else false + } + + override def getAcls(resource: Resource): Set[Acl] = { + inReadLock(lock) { + aclCache.get(resource).getOrElse(Set.empty[Acl]) + } + } + + private def getAclsFromZk(resource: Resource): Set[Acl] = { + val aclJson = ZkUtils.readDataMaybeNull(zkClient, toResourcePath(resource))._1 + aclJson.map(Acl.fromJson).getOrElse(Set.empty) + } + + override def getAcls(principal: KafkaPrincipal): Map[Resource, Set[Acl]] = { + aclCache.mapValues { acls => + acls.filter(_.principal == principal) + }.filter { case (_, acls) => + acls.nonEmpty + }.toMap + } + + override def getAcls(): Map[Resource, Set[Acl]] = { + aclCache.toMap + } + + private def loadCache() { + var acls = Set.empty[Acl] + val resourceTypes = ZkUtils.getChildren(zkClient, SimpleAclAuthorizer.AclZkPath) + for (rType <- resourceTypes) { + val resourceType = ResourceType.fromString(rType) + val resourceTypePath = SimpleAclAuthorizer.AclZkPath + "/" + resourceType.name + val resourceNames = ZkUtils.getChildren(zkClient, resourceTypePath) + for (resourceName <- resourceNames) { + acls = getAclsFromZk(Resource(resourceType, resourceName.toString)) + updateCache(new Resource(resourceType, resourceName), acls) + } + } + } + + private def updateCache(resource: Resource, acls: Set[Acl]) { + inWriteLock(lock) { + if (acls.nonEmpty) + aclCache.put(resource, acls) + else + aclCache.remove(resource) + } + } + + def toResourcePath(resource: Resource): String = { + SimpleAclAuthorizer.AclZkPath + "/" + resource.resourceType + "/" + resource.name + } + + private def logAuditMessage(principal: KafkaPrincipal, authorized: Boolean, operation: Operation, resource: Resource, host: String) { + val permissionType = if (authorized) "Allowed" else "Denied" + authorizerLogger.debug(s"Principal = $principal is $permissionType Operation = $operation from host = $host on resource = $resource") + } + + private def updateAclChangedFlag(resource: Resource) { + ZkUtils.createSequentialPersistentPath(zkClient, SimpleAclAuthorizer.AclChangedZkPath + "/" + SimpleAclAuthorizer.AclChangedPrefix, resource.toString) + } + + object AclChangedNotificaitonHandler extends NotificationHandler { + + override def processNotification(notificationMessage: String) { + val resource: Resource = Resource.fromString(notificationMessage) + val acls = getAclsFromZk(resource) + updateCache(resource, acls) + } + } + + object ZkStateChangeListener extends IZkStateListener { + + override def handleNewSession() { + aclChangeListener.processAllNotifications + } + + override def handleSessionEstablishmentError(error: Throwable) { + fatal("Could not establish session with zookeeper", error) + } + + override def handleStateChanged(state: KeeperState) { + //no op + } + } + +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/serializer/Decoder.scala b/core/src/main/scala/kafka/serializer/Decoder.scala index 54d0b935704ae..164c3fa419c03 100644 --- a/core/src/main/scala/kafka/serializer/Decoder.scala +++ b/core/src/main/scala/kafka/serializer/Decoder.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 @@ -36,16 +36,16 @@ class DefaultDecoder(props: VerifiableProperties = null) extends Decoder[Array[B } /** - * The string encoder translates strings into bytes. It uses UTF8 by default but takes + * The string decoder translates bytes into strings. It uses UTF8 by default but takes * an optional property serializer.encoding to control this. */ class StringDecoder(props: VerifiableProperties = null) extends Decoder[String] { - val encoding = + val encoding = if(props == null) - "UTF8" + "UTF8" else props.getString("serializer.encoding", "UTF8") - + def fromBytes(bytes: Array[Byte]): String = { new String(bytes, encoding) } 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 2e9532e820b5b..21c7e3efe026b --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -17,20 +17,19 @@ package kafka.server -import kafka.cluster.Broker -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.message.{InvalidMessageException, ByteBufferMessageSet, MessageAndOffset} -import kafka.metrics.KafkaMetricsGroup +import java.util.concurrent.locks.ReentrantLock -import scala.collection.mutable -import scala.collection.Set -import scala.collection.Map +import kafka.cluster.BrokerEndPoint +import kafka.consumer.PartitionTopicInfo +import kafka.message.{InvalidMessageException, MessageAndOffset, ByteBufferMessageSet} +import kafka.utils.{Pool, ShutdownableThread, DelayedItem} +import kafka.common.{KafkaException, ClientIdAndBroker, TopicAndPartition} +import kafka.metrics.KafkaMetricsGroup +import kafka.utils.CoreUtils.inLock +import org.apache.kafka.common.protocol.Errors +import AbstractFetcherThread._ +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 @@ -38,29 +37,28 @@ 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, + fetchBackOffMs: Int = 0, isInterruptible: Boolean = true) extends ShutdownableThread(name, isInterruptible) { - private val partitionMap = new mutable.HashMap[TopicAndPartition, Long] // a (topic, partition) -> offset map + + type REQ <: FetchRequest + type PD <: PartitionData + + 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(). - clientId(clientId). - replicaId(fetcherBrokerId). - maxWait(maxWait). - minBytes(minBytes) /* callbacks to be defined in subclass */ // process fetched data - def processPartitionData(topicAndPartition: TopicAndPartition, fetchOffset: Long, - partitionData: FetchResponsePartitionData) + def processPartitionData(topicAndPartition: TopicAndPartition, fetchOffset: Long, partitionData: PD) // handle a partition whose offset is out of range and return a new fetch offset def handleOffsetOutOfRange(topicAndPartition: TopicAndPartition): Long @@ -68,84 +66,93 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke // deal with partitions with errors, potentially due to leadership changes def handlePartitionsWithErrors(partitions: Iterable[TopicAndPartition]) + protected def buildFetchRequest(partitionMap: Map[TopicAndPartition, PartitionFetchState]): REQ + + protected def fetch(fetchRequest: REQ): Map[TopicAndPartition, PD] + override def shutdown(){ - super.shutdown() - simpleConsumer.close() + initiateShutdown() + inLock(partitionMapLock) { + partitionMapCond.signalAll() + } + awaitShutdown() } override def doWork() { - inLock(partitionMapLock) { - if (partitionMap.isEmpty) - partitionMapCond.await(200L, TimeUnit.MILLISECONDS) - partitionMap.foreach { - case((topicAndPartition, offset)) => - fetchRequestBuilder.addFetch(topicAndPartition.topic, topicAndPartition.partition, - offset, fetchSize) + + val fetchRequest = inLock(partitionMapLock) { + val fetchRequest = buildFetchRequest(partitionMap) + if (fetchRequest.isEmpty) { + trace("There are no active partitions. Back off for %d ms before sending a fetch request".format(fetchBackOffMs)) + partitionMapCond.await(fetchBackOffMs, TimeUnit.MILLISECONDS) } + fetchRequest } - val fetchRequest = fetchRequestBuilder.build() - if (!fetchRequest.requestInfo.isEmpty) + if (!fetchRequest.isEmpty) processFetchRequest(fetchRequest) } - private def processFetchRequest(fetchRequest: FetchRequest) { + private def processFetchRequest(fetchRequest: REQ) { val partitionsWithError = new mutable.HashSet[TopicAndPartition] - var response: FetchResponse = null + var responseData: Map[TopicAndPartition, PD] = Map.empty + try { trace("Issuing to broker %d of fetch request %s".format(sourceBroker.id, fetchRequest)) - response = simpleConsumer.fetch(fetchRequest) + responseData = fetch(fetchRequest) } catch { case t: Throwable => if (isRunning.get) { warn("Error in fetch %s. Possible cause: %s".format(fetchRequest, t.toString)) - partitionMapLock synchronized { + inLock(partitionMapLock) { partitionsWithError ++= partitionMap.keys + // there is an error occurred while fetching partitions, sleep a while + partitionMapCond.await(fetchBackOffMs, TimeUnit.MILLISECONDS) } } } fetcherStats.requestRate.mark() - if (response != null) { + if (responseData.nonEmpty) { // process fetched data inLock(partitionMapLock) { - response.data.foreach { - case(topicAndPartition, partitionData) => - val (topic, partitionId) = topicAndPartition.asTuple - val currentOffset = partitionMap.get(topicAndPartition) + + responseData.foreach { case (topicAndPartition, partitionData) => + val TopicAndPartition(topic, partitionId) = topicAndPartition + 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 (currentOffset.isDefined && fetchRequest.requestInfo(topicAndPartition).offset == currentOffset.get) { - partitionData.error match { - case ErrorMapping.NoError => + if (fetchRequest.offset(topicAndPartition) == currentPartitionFetchState.offset) { + Errors.forCode(partitionData.errorCode) match { + case Errors.NONE => try { - val messages = partitionData.messages.asInstanceOf[ByteBufferMessageSet] + val messages = partitionData.toByteBufferMessageSet val validBytes = messages.validBytes val newOffset = messages.shallowIterator.toSeq.lastOption match { case Some(m: MessageAndOffset) => m.nextOffset - case None => currentOffset.get + case None => currentPartitionFetchState.offset } - partitionMap.put(topicAndPartition, newOffset) - fetcherLagStats.getFetcherLagStats(topic, partitionId).lag = partitionData.hw - newOffset + partitionMap.put(topicAndPartition, new PartitionFetchState(newOffset)) + fetcherLagStats.getFetcherLagStats(topic, partitionId).lag = partitionData.highWatermark - 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) + 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 " + currentOffset.get + " error " + ime.getMessage) + // 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, currentOffset.get), e) + .format(topic, partitionId, currentPartitionFetchState.offset), e) } - case ErrorMapping.OffsetOutOfRangeCode => + case Errors.OFFSET_OUT_OF_RANGE => try { val newOffset = handleOffsetOutOfRange(topicAndPartition) - partitionMap.put(topicAndPartition, newOffset) + partitionMap.put(topicAndPartition, new PartitionFetchState(newOffset)) error("Current offset %d for partition [%s,%d] out of range; reset offset to %d" - .format(currentOffset.get, topic, partitionId, newOffset)) + .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) @@ -154,16 +161,16 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke case _ => if (isRunning.get) { error("Error for partition [%s,%d] to broker %d:%s".format(topic, partitionId, sourceBroker.id, - ErrorMapping.exceptionFor(partitionData.error).getClass)) + partitionData.exception.get)) partitionsWithError += topicAndPartition } } - } + }) } } } - if(partitionsWithError.size > 0) { + if (partitionsWithError.nonEmpty) { debug("handling partitions with error for %s".format(partitionsWithError)) handlePartitionsWithErrors(partitionsWithError) } @@ -177,40 +184,65 @@ 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 { - partitionMapLock.unlock() - } + } finally partitionMapLock.unlock() } def removePartitions(topicAndPartitions: Set[TopicAndPartition]) { partitionMapLock.lockInterruptibly() - try { - topicAndPartitions.foreach(tp => partitionMap.remove(tp)) - } finally { - partitionMapLock.unlock() - } + try topicAndPartitions.foreach(partitionMap.remove) + finally partitionMapLock.unlock() } def partitionCount() = { partitionMapLock.lockInterruptibly() - try { - partitionMap.size - } finally { - partitionMapLock.unlock() - } + try partitionMap.size + finally partitionMapLock.unlock() } + } -class FetcherLagMetrics(metricId: ClientIdBrokerTopicPartition) extends KafkaMetricsGroup { +object AbstractFetcherThread { + + trait FetchRequest { + def isEmpty: Boolean + def offset(topicAndPartition: TopicAndPartition): Long + } + + trait PartitionData { + def errorCode: Short + def exception: Option[Throwable] + def toByteBufferMessageSet: ByteBufferMessageSet + def highWatermark: Long + } + +} + +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) { @@ -221,20 +253,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/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala new file mode 100644 index 0000000000000..24f294d2b43df --- /dev/null +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -0,0 +1,267 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.{DelayQueue, TimeUnit} + +import kafka.utils.{ShutdownableThread, Logging} +import org.apache.kafka.common.MetricName +import org.apache.kafka.common.metrics._ +import org.apache.kafka.common.metrics.stats.{Total, Rate, Avg} +import java.util.concurrent.locks.ReentrantReadWriteLock + +import org.apache.kafka.common.utils.Time + +/** + * Represents the sensors aggregated per client + * @param quotaSensor @Sensor that tracks the quota + * @param throttleTimeSensor @Sensor that tracks the throttle time + */ +private case class ClientSensors(quotaSensor: Sensor, throttleTimeSensor: Sensor) + +/** + * Configuration settings for quota management + * @param quotaBytesPerSecondDefault The default bytes per second quota allocated to any client + * @param quotaBytesPerSecondOverrides The comma separated overrides per client. "c1=X,c2=Y" + * @param numQuotaSamples The number of samples to retain in memory + * @param quotaWindowSizeSeconds The time span of each sample + * + */ +case class ClientQuotaManagerConfig(quotaBytesPerSecondDefault: Long = + ClientQuotaManagerConfig.QuotaBytesPerSecondDefault, + quotaBytesPerSecondOverrides: String = + ClientQuotaManagerConfig.QuotaBytesPerSecondOverrides, + numQuotaSamples: Int = + ClientQuotaManagerConfig.DefaultNumQuotaSamples, + quotaWindowSizeSeconds: Int = + ClientQuotaManagerConfig.DefaultQuotaWindowSizeSeconds) + +object ClientQuotaManagerConfig { + val QuotaBytesPerSecondDefault = Long.MaxValue + val QuotaBytesPerSecondOverrides = "" + // Always have 10 whole windows + 1 current window + val DefaultNumQuotaSamples = 11 + val DefaultQuotaWindowSizeSeconds = 1 + val MaxThrottleTimeSeconds = 30 + // Purge sensors after 1 hour of inactivity + val InactiveSensorExpirationTimeSeconds = 3600 +} + +/** + * Helper class that records per-client metrics. It is also responsible for maintaining Quota usage statistics + * for all clients. + * @param config @ClientQuotaManagerConfig quota configs + * @param metrics @Metrics Metrics instance + * @param apiKey API Key for the request + * @param time @Time object to use + */ +class ClientQuotaManager(private val config: ClientQuotaManagerConfig, + private val metrics: Metrics, + private val apiKey: String, + private val time: Time) extends Logging { + private val overriddenQuota = initQuotaMap(config.quotaBytesPerSecondOverrides) + private val defaultQuota = Quota.lessThan(config.quotaBytesPerSecondDefault) + private val lock = new ReentrantReadWriteLock() + private val delayQueue = new DelayQueue[ThrottledResponse]() + val throttledRequestReaper = new ThrottledRequestReaper(delayQueue) + throttledRequestReaper.start() + + private val delayQueueSensor = metrics.sensor(apiKey + "-delayQueue") + delayQueueSensor.add(new MetricName("queue-size", + apiKey, + "Tracks the size of the delay queue"), new Total()) + + /** + * Reaper thread that triggers callbacks on all throttled requests + * @param delayQueue DelayQueue to dequeue from + */ + class ThrottledRequestReaper(delayQueue: DelayQueue[ThrottledResponse]) extends ShutdownableThread( + "ThrottledRequestReaper-%s".format(apiKey), false) { + + override def doWork(): Unit = { + val response: ThrottledResponse = delayQueue.poll(1, TimeUnit.SECONDS) + if (response != null) { + // Decrement the size of the delay queue + delayQueueSensor.record(-1) + trace("Response throttled for: " + response.throttleTimeMs + " ms") + response.execute() + } + } + } + + /** + * Records that a clientId changed some metric being throttled (produced/consumed bytes, QPS etc.) + * @param clientId clientId that produced the data + * @param value amount of data written in bytes + * @param callback Callback function. This will be triggered immediately if quota is not violated. + * If there is a quota violation, this callback will be triggered after a delay + * @return Number of milliseconds to delay the response in case of Quota violation. + * Zero otherwise + */ + def recordAndMaybeThrottle(clientId: String, value: Int, callback: Int => Unit): Int = { + val clientSensors = getOrCreateQuotaSensors(clientId) + var throttleTimeMs = 0 + try { + clientSensors.quotaSensor.record(value) + // trigger the callback immediately if quota is not violated + callback(0) + } catch { + case qve: QuotaViolationException => + // Compute the delay + val clientMetric = metrics.metrics().get(clientRateMetricName(clientId)) + throttleTimeMs = throttleTime(clientMetric, getQuotaMetricConfig(quota(clientId))) + delayQueue.add(new ThrottledResponse(time, throttleTimeMs, callback)) + delayQueueSensor.record() + // If delayed, add the element to the delayQueue + logger.debug("Quota violated for sensor (%s). Delay time: (%d)".format(clientSensors.quotaSensor.name(), throttleTimeMs)) + } + // If the request is not throttled, a throttleTime of 0 ms is recorded + clientSensors.throttleTimeSensor.record(throttleTimeMs) + throttleTimeMs + } + + /* + * This calculates the amount of time needed to bring the metric within quota + * assuming that no new metrics are recorded. + * + * Basically, if O is the observed rate and T is the target rate over a window of W, to bring O down to T, + * we need to add a delay of X to W such that O * W / (W + X) = T. + * Solving for X, we get X = (O - T)/T * W. + */ + private def throttleTime(clientMetric: KafkaMetric, config: MetricConfig): Int = { + val rateMetric: Rate = measurableAsRate(clientMetric.metricName(), clientMetric.measurable()) + val quota = config.quota() + val difference = clientMetric.value() - quota.bound + // Use the precise window used by the rate calculation + val throttleTimeMs = difference / quota.bound * rateMetric.windowSize(config, time.milliseconds()) + throttleTimeMs.round.toInt + } + + // Casting to Rate because we only use Rate in Quota computation + private def measurableAsRate(name: MetricName, measurable: Measurable): Rate = { + measurable match { + case r: Rate => r + case _ => throw new IllegalArgumentException(s"Metric $name is not a Rate metric, value $measurable") + } + } + + /** + * Returns the consumer quota for the specified clientId + * @return + */ + private[server] def quota(clientId: String): Quota = overriddenQuota.getOrElse(clientId, defaultQuota) + + /* + * This function either returns the sensors for a given client id or creates them if they don't exist + * First sensor of the tuple is the quota enforcement sensor. Second one is the throttle time sensor + */ + private def getOrCreateQuotaSensors(clientId: String): ClientSensors = { + + // Names of the sensors to access + val quotaSensorName = apiKey + "-" + clientId + val throttleTimeSensorName = apiKey + "ThrottleTime-" + clientId + var quotaSensor: Sensor = null + var throttleTimeSensor: Sensor = null + + /* Acquire the read lock to fetch the sensors. It is safe to call getSensor from multiple threads. + * The read lock allows a thread to create a sensor in isolation. The thread creating the sensor + * will acquire the write lock and prevent the sensors from being read while they are being created. + * It should be sufficient to simply check if the sensor is null without acquiring a read lock but the + * sensor being present doesn't mean that it is fully initialized i.e. all the Metrics may not have been added. + * This read lock waits until the writer thread has released its lock i.e. fully initialized the sensor + * at which point it is safe to read + */ + lock.readLock().lock() + try { + quotaSensor = metrics.getSensor(quotaSensorName) + throttleTimeSensor = metrics.getSensor(throttleTimeSensorName) + } + finally { + lock.readLock().unlock() + } + + /* If the sensor is null, try to create it else return the created sensor + * Also if quota sensor is null, the throttle time sensor must be null + */ + if (quotaSensor == null) { + /* Acquire a write lock because the sensor may not have been created and we only want one thread to create it. + * Note that multiple threads may acquire the write lock if they all see a null sensor initially + * In this case, the writer checks the sensor after acquiring the lock again. + * This is safe from Double Checked Locking because the references are read + * after acquiring read locks and hence they cannot see a partially published reference + */ + lock.writeLock().lock() + try { + quotaSensor = metrics.getSensor(quotaSensorName) + if (quotaSensor == null) { + // create the throttle time sensor also. Use default metric config + throttleTimeSensor = metrics.sensor(throttleTimeSensorName, + null, + ClientQuotaManagerConfig.InactiveSensorExpirationTimeSeconds) + throttleTimeSensor.add(new MetricName("throttle-time", + apiKey, + "Tracking average throttle-time per client", + "client-id", + clientId), new Avg()) + + quotaSensor = metrics.sensor(quotaSensorName, + getQuotaMetricConfig(quota(clientId)), + ClientQuotaManagerConfig.InactiveSensorExpirationTimeSeconds) + quotaSensor.add(clientRateMetricName(clientId), new Rate()) + } + } finally { + lock.writeLock().unlock() + } + } + // return the read or created sensors + ClientSensors(quotaSensor, throttleTimeSensor) + } + + private def getQuotaMetricConfig(quota: Quota): MetricConfig = { + new MetricConfig() + .timeWindow(config.quotaWindowSizeSeconds, TimeUnit.SECONDS) + .samples(config.numQuotaSamples) + .quota(quota) + } + + /* Construct a Map of (clientId -> Quota) + * The input config is specified as a comma-separated K=V pairs + */ + private def initQuotaMap(input: String): Map[String, Quota] = { + // If empty input, return an empty map + if (input.trim.length == 0) + Map[String, Quota]() + else + input.split(",").map(entry => { + val trimmedEntry = entry.trim + val pair: Array[String] = trimmedEntry.split("=") + if (pair.length != 2) + throw new IllegalArgumentException("Incorrectly formatted override entry (%s). Format is k1=v1,k2=v2".format(entry)) + pair(0) -> new Quota(pair(1).toDouble, true) + }).toMap + } + + private def clientRateMetricName(clientId: String): MetricName = { + new MetricName("byte-rate", apiKey, + "Tracking byte-rate per client", + "client-id", clientId) + } + + def shutdown() = { + throttledRequestReaper.shutdown() + } +} diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala new file mode 100644 index 0000000000000..8347a69a34c38 --- /dev/null +++ b/core/src/main/scala/kafka/server/ConfigHandler.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 java.util.Properties + +import kafka.common.TopicAndPartition +import kafka.log.{Log, LogConfig, LogManager} +import kafka.utils.Pool + +import scala.collection.mutable + +/** + * The ConfigHandler is used to process config change notifications received by the DynamicConfigManager + */ +trait ConfigHandler { + def processConfigChanges(entityName : String, value : Properties) +} + +/** + * The TopicConfigHandler will process topic config changes in ZK. + * The callback provides the topic name and the full properties set read from ZK + */ +class TopicConfigHandler(private val logManager: LogManager) extends ConfigHandler{ + + def processConfigChanges(topic : String, topicConfig : Properties) { + val logs: mutable.Buffer[(TopicAndPartition, Log)] = logManager.logsByTopicPartition.toBuffer + val logsByTopic: Map[String, mutable.Buffer[Log]] = logs.groupBy{ case (topicAndPartition, log) => topicAndPartition.topic } + .mapValues{ case v: mutable.Buffer[(TopicAndPartition, Log)] => v.map(_._2) } + + 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(topicConfig) + val logConfig = LogConfig(props) + for (log <- logsByTopic(topic)) + log.config = logConfig + } + } +} + +/** + * The ClientIdConfigHandler will process clientId config changes in ZK. + * The callback provides the clientId and the full properties set read from ZK. + * This implementation does nothing currently. In the future, it will change quotas per client + */ +class ClientIdConfigHandler extends ConfigHandler { + val configPool = new Pool[String, Properties]() + + def processConfigChanges(clientId : String, clientConfig : Properties): Unit = { + configPool.put(clientId, clientConfig) + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index e0f14e25af03e..de6cf5bdaa0e7 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -17,75 +17,127 @@ package kafka.server -import kafka.network.RequestChannel -import kafka.api.{FetchResponse, FetchRequest} -import kafka.common.{UnknownTopicOrPartitionException, NotLeaderForPartitionException, TopicAndPartition} +import java.util.concurrent.TimeUnit -import scala.collection.immutable.Map -import scala.collection.Seq +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 + "]" +} /** - * A delayed fetch request, which is satisfied (or more - * accurately, unblocked) -- if: - * Case A: This broker is no longer the leader for some partitions it tries to fetch - * - should return whatever data is available for the rest partitions. - * Case B: This broker is does not know of some partitions it tries to fetch - * - should return whatever data is available for the rest partitions. - * Case C: The fetch offset locates not on the last segment of the log - * - should return all the data on that segment. - * Case D: The accumulated bytes from all the fetching partitions exceeds the minimum bytes - * - should return whatever data is available. + * The fetch metadata maintained by the delayed fetch operation */ +case class FetchMetadata(fetchMinBytes: Int, + fetchOnlyLeader: Boolean, + fetchOnlyCommitted: Boolean, + isFromFollower: Boolean, + fetchPartitionStatus: Map[TopicAndPartition, FetchPartitionStatus]) { -class DelayedFetch(override val keys: Seq[TopicPartitionRequestKey], - override val request: RequestChannel.Request, - override val delayMs: Long, - val fetch: FetchRequest, - private val partitionFetchOffsets: Map[TopicAndPartition, LogOffsetMetadata]) - extends DelayedRequest(keys, request, delayMs) { + 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) { - def isSatisfied(replicaManager: ReplicaManager) : Boolean = { + /** + * 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 - val fromFollower = fetch.isFromFollower - partitionFetchOffsets.foreach { - case (topicAndPartition, fetchOffset) => + 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 (fromFollower) - replica.logEndOffset - else + if (fetchMetadata.fetchOnlyCommitted) replica.highWatermark + else + replica.logEndOffset if (endOffset.offsetOnOlderSegment(fetchOffset)) { - // Case C, this can happen when the new follower replica fetching on a truncated leader - debug("Satisfying fetch request %s since it is fetching later segments of partition %s.".format(fetch, topicAndPartition)) - return true + // 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 folloer replica is lagging too much - debug("Satisfying fetch request %s immediately since it is fetching older segments.".format(fetch)) - return true + // 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)) { - accumulatedSize += endOffset.positionDiff(fetchOffset) + // 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 A - debug("Broker no longer know of %s, satisfy %s immediately".format(topicAndPartition, fetch)) - return true - case nle: NotLeaderForPartitionException => // Case B - debug("Broker is no longer the leader of %s, satisfy %s immediately".format(topicAndPartition, fetch)) - return true + 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 - accumulatedSize >= fetch.minBytes + if (accumulatedSize >= fetchMetadata.fetchMinBytes) + forceComplete() + else + false } - def respond(replicaManager: ReplicaManager): FetchResponse = { - val topicData = replicaManager.readMessageSets(fetch) - FetchResponse(fetch.correlationId, topicData.mapValues(_.data)) + override def onExpiration() { + if (fetchMetadata.isFromFollower) + DelayedFetchMetrics.followerExpiredRequestMeter.mark() + else + DelayedFetchMetrics.consumerExpiredRequestMeter.mark() } -} \ No newline at end of file + + /** + * 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/DelayedRequestKey.scala b/core/src/main/scala/kafka/server/DelayedOperationKey.scala similarity index 63% rename from core/src/main/scala/kafka/server/DelayedRequestKey.scala rename to core/src/main/scala/kafka/server/DelayedOperationKey.scala index 628ef59564b9b..c122bdebc6b90 100644 --- a/core/src/main/scala/kafka/server/DelayedRequestKey.scala +++ b/core/src/main/scala/kafka/server/DelayedOperationKey.scala @@ -20,19 +20,32 @@ package kafka.server import kafka.common.TopicAndPartition /** - * Keys used for delayed request metrics recording + * Keys used for delayed operation metrics recording */ -trait DelayedRequestKey { +trait DelayedOperationKey { def keyLabel: String } -object DelayedRequestKey { +object DelayedOperationKey { val globalLabel = "All" } -case class TopicPartitionRequestKey(topic: String, partition: Int) extends DelayedRequestKey { +/* 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 index 9481508fc2d61..05078b24ef28f 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -17,99 +17,133 @@ package kafka.server -import kafka.api._ + +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.utils.Logging -import kafka.network.RequestChannel - -import scala.Some -import scala.collection.immutable.Map -import scala.collection.Seq - -/** A delayed produce request, which is satisfied (or more - * accurately, unblocked) -- if for every partition it produce to: - * 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. - */ - -class DelayedProduce(override val keys: Seq[TopicPartitionRequestKey], - override val request: RequestChannel.Request, - override val delayMs: Long, - val produce: ProducerRequest, - val partitionStatus: Map[TopicAndPartition, DelayedProduceResponseStatus], - val offsetCommitRequestOpt: Option[OffsetCommitRequest] = None) - extends DelayedRequest(keys, request, delayMs) with Logging { +import kafka.metrics.KafkaMetricsGroup +import kafka.utils.Pool - // first update the acks pending variable according to the error code - partitionStatus foreach { case (topicAndPartition, delayedStatus) => - if (delayedStatus.responseStatus.error == ErrorMapping.NoError) { - // Timeout error state will be cleared when required acks are received - delayedStatus.acksPending = true - delayedStatus.responseStatus.error = ErrorMapping.RequestTimedOutCode - } else { - delayedStatus.acksPending = false - } +import scala.collection._ - trace("Initial partition status for %s is %s".format(topicAndPartition, delayedStatus)) - } +case class ProducePartitionStatus(requiredOffset: Long, responseStatus: ProducerResponseStatus) { + @volatile var acksPending = false - def respond(offsetManager: OffsetManager): RequestOrResponse = { - val responseStatus = partitionStatus.mapValues(status => status.responseStatus) + override def toString = "[acksPending: %b, error: %d, startOffset: %d, requiredOffset: %d]" + .format(acksPending, responseStatus.error, responseStatus.offset, requiredOffset) +} - val errorCode = responseStatus.find { case (_, status) => - status.error != ErrorMapping.NoError - }.map(_._2.error).getOrElse(ErrorMapping.NoError) +/** + * The produce metadata maintained by the delayed produce operation + */ +case class ProduceMetadata(produceRequiredAcks: Short, + produceStatus: Map[TopicAndPartition, ProducePartitionStatus]) { - if (errorCode == ErrorMapping.NoError) { - offsetCommitRequestOpt.foreach(ocr => offsetManager.putOffsets(ocr.groupId, ocr.requestInfo) ) - } + override def toString = "[requiredAcks: %d, partitionStatus: %s]" + .format(produceRequiredAcks, produceStatus) +} - val response = offsetCommitRequestOpt.map(_.responseFor(errorCode, offsetManager.config.maxMetadataSize)) - .getOrElse(ProducerResponse(produce.correlationId, responseStatus)) +/** + * 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) { - response + // 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)) } - def isSatisfied(replicaManager: ReplicaManager) = { + /** + * 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 - partitionStatus.foreach { case (topicAndPartition, fetchPartitionStatus) => - trace("Checking producer request satisfaction for %s, acksPending = %b" - .format(topicAndPartition, fetchPartitionStatus.acksPending)) + 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 (fetchPartitionStatus.acksPending) { + if (status.acksPending) { val partitionOpt = replicaManager.getPartition(topicAndPartition.topic, topicAndPartition.partition) val (hasEnough, errorCode) = partitionOpt match { case Some(partition) => - partition.checkEnoughReplicasReachOffset( - fetchPartitionStatus.requiredOffset, - produce.requiredAcks) + partition.checkEnoughReplicasReachOffset(status.requiredOffset) case None => + // Case A (false, ErrorMapping.UnknownTopicOrPartitionCode) } if (errorCode != ErrorMapping.NoError) { - fetchPartitionStatus.acksPending = false - fetchPartitionStatus.responseStatus.error = errorCode + // Case B.1 + status.acksPending = false + status.responseStatus.error = errorCode } else if (hasEnough) { - fetchPartitionStatus.acksPending = false - fetchPartitionStatus.responseStatus.error = ErrorMapping.NoError + // Case B.2 + status.acksPending = false + status.responseStatus.error = ErrorMapping.NoError } } } - // unblocked if there are no partitions with pending acks - val satisfied = ! partitionStatus.exists(p => p._2.acksPending) - satisfied + // 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) } } -case class DelayedProduceResponseStatus(val requiredOffset: Long, - val responseStatus: ProducerResponseStatus) { - @volatile var acksPending = false +object DelayedProduceMetrics extends KafkaMetricsGroup { + + private val aggregateExpirationMeter = newMeter("ExpiresPerSec", "requests", TimeUnit.SECONDS) - override def toString = - "acksPending:%b, error: %d, startOffset: %d, requiredOffset: %d".format( - acksPending, responseStatus.error, responseStatus.offset, requiredOffset) + 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/server/TopicConfigManager.scala b/core/src/main/scala/kafka/server/DynamicConfigManager.scala similarity index 50% rename from core/src/main/scala/kafka/server/TopicConfigManager.scala rename to core/src/main/scala/kafka/server/DynamicConfigManager.scala index 47295d4013149..4da1833a3d3ed 100644 --- a/core/src/main/scala/kafka/server/TopicConfigManager.scala +++ b/core/src/main/scala/kafka/server/DynamicConfigManager.scala @@ -17,67 +17,79 @@ package kafka.server -import java.util.Properties +import kafka.utils.Json +import kafka.utils.Logging +import kafka.utils.SystemTime +import kafka.utils.Time +import kafka.utils.ZkUtils + import scala.collection._ -import kafka.log._ -import kafka.utils._ import kafka.admin.AdminUtils import org.I0Itec.zkclient.{IZkChildListener, ZkClient} + +/** + * Represents all the entities that can be configured via ZK + */ +object ConfigType { + val Topic = "topics" + val Client = "clients" +} + /** - * This class initiates and carries out topic config changes. - * + * This class initiates and carries out config changes for all entities defined in ConfigType. + * * It works as follows. - * - * Config is stored under the path - * /brokers/topics//config - * This znode stores the topic-overrides for this topic (but no defaults) in properties format. - * + * + * Config is stored under the path: /config/entityType/entityName + * E.g. /config/topics/ and /config/clients/ + * This znode stores the overrides for this entity (but no defaults) in properties format. + * * To avoid watching all topics for changes instead we have a notification path - * /brokers/config_changes - * The TopicConfigManager has a child watch on this path. - * - * To update a topic config we first update the topic config properties. Then we create a new sequential - * znode under the change path which contains the name of the topic that was updated, say - * /brokers/config_changes/config_change_13321 - * This is just a notification--the actual config change is stored only once under the /brokers/topics//config path. - * + * /config/changes + * The DynamicConfigManager has a child watch on this path. + * + * To update a config we first update the config properties. Then we create a new sequential + * znode under the change path which contains the name of the entityType and entityName that was updated, say + * /config/changes/config_change_13321 + * The sequential znode contains data in this format: {"version" : 1, "entityType":"topic/client", "entityName" : "topic_name/client_id"} + * This is just a notification--the actual config change is stored only once under the /config/entityType/entityName 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 - * it checks if this notification is larger than a static expiration time (say 10mins) and if so it deletes this notification. - * For any new changes it reads the new configuration, combines it with the defaults, and updates the log config - * for all logs for that topic (if any) that it has. - * + * it checks if this notification is larger than a static expiration time (say 10mins) and if so it deletes this notification. + * For any new changes it reads the new configuration, combines it with the defaults, and updates the existing config. + * * Note that config is always read from the config path in zk, the notification is just a trigger to do so. So if a broker is * down and misses a change that is fine--when it restarts it will be loading the full config anyway. Note also that - * if there are two consecutive config changes it is possible that only the last one will be applied (since by the time the + * if there are two consecutive config changes it is possible that only the last one will be applied (since by the time the * broker reads the config the both changes may have been made). In this case the broker would needlessly refresh the config twice, * but that is harmless. - * + * * On restart the config manager re-processes all notifications. This will usually be wasted work, but avoids any race conditions * on startup where a change might be missed between the initial config load and registering for change notifications. - * + * */ -class TopicConfigManager(private val zkClient: ZkClient, - private val logManager: LogManager, - private val changeExpirationMs: Long = 15*60*1000, - private val time: Time = SystemTime) extends Logging { +class DynamicConfigManager(private val zkClient: ZkClient, + private val configHandler : Map[String, ConfigHandler], + private val changeExpirationMs: Long = 15*60*1000, + private val time: Time = SystemTime) extends Logging { private var lastExecutedChange = -1L - + /** * Begin watching for config changes */ def startup() { - ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.TopicConfigChangesPath) - zkClient.subscribeChildChanges(ZkUtils.TopicConfigChangesPath, ConfigChangeListener) + ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.EntityConfigChangesPath) + zkClient.subscribeChildChanges(ZkUtils.EntityConfigChangesPath, ConfigChangeListener) processAllConfigChanges() } - + /** * Process all config changes */ private def processAllConfigChanges() { - val configChanges = zkClient.getChildren(ZkUtils.TopicConfigChangesPath) + val configChanges = zkClient.getChildren(ZkUtils.EntityConfigChangesPath) import JavaConversions._ processConfigChanges((configChanges: mutable.Buffer[String]).sorted) } @@ -89,38 +101,59 @@ class TopicConfigManager(private val zkClient: ZkClient, if (notifications.size > 0) { info("Processing config change notification(s)...") val now = time.milliseconds - val logs = logManager.logsByTopicPartition.toBuffer - val logsByTopic = logs.groupBy(_._1.topic).mapValues(_.map(_._2)) for (notification <- notifications) { val changeId = changeNumber(notification) + if (changeId > lastExecutedChange) { - val changeZnode = ZkUtils.TopicConfigChangesPath + "/" + notification + val changeZnode = ZkUtils.EntityConfigChangesPath + "/" + notification + 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(logManager.defaultConfig.toProps) - props.putAll(AdminUtils.fetchTopicConfig(zkClient, topic)) - val logConfig = LogConfig.fromProps(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) - } - } - lastExecutedChange = changeId + processNotification(jsonOpt) } + lastExecutedChange = changeId } + purgeObsoleteNotifications(now, notifications) } } - + + def processNotification(jsonOpt: Option[String]) = { + if(jsonOpt.isDefined) { + val json = jsonOpt.get + Json.parseFull(json) match { + case None => // There are no config overrides. + // Ignore non-json notifications because they can be from the deprecated TopicConfigManager + case Some(mapAnon: Map[_, _]) => + val map = mapAnon collect + { case (k: String, v: Any) => k -> v } + require(map("version") == 1) + + val entityType = map.get("entity_type") match { + case Some(ConfigType.Topic) => ConfigType.Topic + case Some(ConfigType.Client) => ConfigType.Client + case _ => throw new IllegalArgumentException("Config change notification must have 'entity_type' set to either 'client' or 'topic'." + + " Received: " + json) + } + + val entity = map.get("entity_name") match { + case Some(value: String) => value + case _ => throw new IllegalArgumentException("Config change notification does not specify 'entity_name'. Received: " + json) + } + configHandler(entityType).processConfigChanges(entity, AdminUtils.fetchEntityConfig(zkClient, entityType, entity)) + + case o => throw new IllegalArgumentException("Config change notification has an unexpected value. The format is:" + + "{\"version\" : 1," + + " \"entity_type\":\"topic/client\"," + + " \"entity_name\" : \"topic_name/client_id\"}." + + " Received: " + json) + } + } + } + private def purgeObsoleteNotifications(now: Long, notifications: Seq[String]) { for(notification <- notifications.sorted) { - val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(zkClient, ZkUtils.TopicConfigChangesPath + "/" + notification) + val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(zkClient, ZkUtils.EntityConfigChangesPath + "/" + notification) if(jsonOpt.isDefined) { - val changeZnode = ZkUtils.TopicConfigChangesPath + "/" + notification + val changeZnode = ZkUtils.EntityConfigChangesPath + "/" + notification if (now - stat.getCtime > changeExpirationMs) { debug("Purging config change notification " + notification) ZkUtils.deletePath(zkClient, changeZnode) @@ -130,10 +163,10 @@ class TopicConfigManager(private val zkClient: ZkClient, } } } - + /* get the change number from a change notification znode */ - private def changeNumber(name: String): Long = name.substring(AdminUtils.TopicConfigChangeZnodePrefix.length).toLong - + private def changeNumber(name: String): Long = name.substring(AdminUtils.EntityConfigChangeZnodePrefix.length).toLong + /** * A listener that applies config changes to logs */ @@ -147,5 +180,4 @@ class TopicConfigManager(private val zkClient: ZkClient, } } } - -} \ No newline at end of file +} diff --git a/core/src/main/scala/kafka/server/FetchDataInfo.scala b/core/src/main/scala/kafka/server/FetchDataInfo.scala index 26f278f9b75b1..1a8a60494100e 100644 --- a/core/src/main/scala/kafka/server/FetchDataInfo.scala +++ b/core/src/main/scala/kafka/server/FetchDataInfo.scala @@ -19,4 +19,4 @@ package kafka.server import kafka.message.MessageSet -case class FetchDataInfo(fetchOffset: LogOffsetMetadata, messageSet: MessageSet) +case class FetchDataInfo(fetchOffsetMetadata: LogOffsetMetadata, messageSet: MessageSet) diff --git a/core/src/main/scala/kafka/server/FetchRequestPurgatory.scala b/core/src/main/scala/kafka/server/FetchRequestPurgatory.scala deleted file mode 100644 index ed13188912535..0000000000000 --- a/core/src/main/scala/kafka/server/FetchRequestPurgatory.scala +++ /dev/null @@ -1,69 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.server - -import kafka.metrics.KafkaMetricsGroup -import kafka.network.RequestChannel -import kafka.api.FetchResponseSend - -import java.util.concurrent.TimeUnit - -/** - * The purgatory holding delayed fetch requests - */ -class FetchRequestPurgatory(replicaManager: ReplicaManager, requestChannel: RequestChannel) - extends RequestPurgatory[DelayedFetch](replicaManager.config.brokerId, replicaManager.config.fetchPurgatoryPurgeIntervalRequests) { - this.logIdent = "[FetchRequestPurgatory-%d] ".format(replicaManager.config.brokerId) - - 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 aggregateFollowerFetchRequestMetrics = new DelayedFetchRequestMetrics(forFollower = true) - private val aggregateNonFollowerFetchRequestMetrics = new DelayedFetchRequestMetrics(forFollower = false) - - private def recordDelayedFetchExpired(forFollower: Boolean) { - val metrics = if (forFollower) aggregateFollowerFetchRequestMetrics - else aggregateNonFollowerFetchRequestMetrics - - metrics.expiredRequestMeter.mark() - } - - /** - * Check if a specified delayed fetch request is satisfied - */ - def checkSatisfied(delayedFetch: DelayedFetch): Boolean = delayedFetch.isSatisfied(replicaManager) - - /** - * When a delayed fetch request expires just answer it with whatever data is present - */ - def expire(delayedFetch: DelayedFetch) { - debug("Expiring fetch request %s.".format(delayedFetch.fetch)) - val fromFollower = delayedFetch.fetch.isFromFollower - recordDelayedFetchExpired(fromFollower) - respond(delayedFetch) - } - - // TODO: purgatory should not be responsible for sending back the responses - def respond(delayedFetch: DelayedFetch) { - val response = delayedFetch.respond(replicaManager) - requestChannel.sendResponse(new RequestChannel.Response(delayedFetch.request, new FetchResponseSend(response))) - } -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index c584b559416b3..72f30441e09bf 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -17,46 +17,53 @@ package kafka.server +import kafka.message.MessageSet +import kafka.security.auth.Topic +import org.apache.kafka.common.metrics.Metrics +import org.apache.kafka.common.protocol.SecurityProtocol +import org.apache.kafka.common.TopicPartition import kafka.api._ +import kafka.admin.AdminUtils import kafka.common._ +import kafka.controller.KafkaController +import kafka.coordinator.ConsumerCoordinator import kafka.log._ -import kafka.message._ import kafka.network._ -import kafka.admin.AdminUtils -import kafka.network.RequestChannel.Response -import kafka.controller.KafkaController -import kafka.utils.{SystemTime, Logging} - +import kafka.network.RequestChannel.{Session, Response} +import org.apache.kafka.common.requests.{JoinGroupRequest, JoinGroupResponse, HeartbeatRequest, HeartbeatResponse, ResponseHeader, ResponseSend} +import kafka.utils.{ZkUtils, ZKGroupTopicDirs, SystemTime, Logging} import scala.collection._ - import org.I0Itec.zkclient.ZkClient +import kafka.security.auth.{Authorizer, Read, Write, Create, ClusterAction, Describe, Resource, Topic, Operation, ConsumerGroup} + /** * Logic to handle the various Kafka requests */ class KafkaApis(val requestChannel: RequestChannel, val replicaManager: ReplicaManager, - val offsetManager: OffsetManager, + val coordinator: ConsumerCoordinator, + val controller: KafkaController, val zkClient: ZkClient, val brokerId: Int, val config: KafkaConfig, - val controller: KafkaController) extends Logging { + val metadataCache: MetadataCache, + val metrics: Metrics, + val authorizer: Option[Authorizer]) extends Logging { - val producerRequestPurgatory = new ProducerRequestPurgatory(replicaManager, offsetManager, requestChannel) - val fetchRequestPurgatory = new FetchRequestPurgatory(replicaManager, requestChannel) - // TODO: the following line will be removed in 0.9 - replicaManager.initWithRequestPurgatory(producerRequestPurgatory, fetchRequestPurgatory) - var metadataCache = new MetadataCache this.logIdent = "[KafkaApi-%d] ".format(brokerId) + // Store all the quota managers for each type of request + private val quotaManagers = instantiateQuotaManagers(config) /** * Top-level method that handles all requests and multiplexes to the right api */ def handle(request: RequestChannel.Request) { try{ - trace("Handling request: " + request.requestObj + " from client: " + request.remoteAddress) + trace("Handling request:%s from connection %s;securityProtocol:%s,principal:%s". + format(request.requestObj, request.connectionId, request.securityProtocol, request.session.principal)) request.requestId match { - case RequestKeys.ProduceKey => handleProducerOrOffsetCommitRequest(request) + case RequestKeys.ProduceKey => handleProducerRequest(request) case RequestKeys.FetchKey => handleFetchRequest(request) case RequestKeys.OffsetsKey => handleOffsetRequest(request) case RequestKeys.MetadataKey => handleTopicMetadataRequest(request) @@ -64,14 +71,28 @@ class KafkaApis(val requestChannel: RequestChannel, case RequestKeys.StopReplicaKey => handleStopReplicaRequest(request) case RequestKeys.UpdateMetadataKey => handleUpdateMetadataRequest(request) case RequestKeys.ControlledShutdownKey => handleControlledShutdownRequest(request) - case RequestKeys.OffsetCommitKey => handleProducerOrOffsetCommitRequest(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 @@ -82,10 +103,28 @@ class KafkaApis(val requestChannel: RequestChannel, // 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] + + authorizeClusterAction(request) + try { - val (response, error) = replicaManager.becomeLeaderOrFollower(leaderAndIsrRequest, offsetManager) - 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) @@ -98,18 +137,24 @@ class KafkaApis(val requestChannel: RequestChannel, // 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 stopReplicaRequest = request.requestObj.asInstanceOf[StopReplicaRequest] + + authorizeClusterAction(request) + 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] + + authorizeClusterAction(request) + 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) { @@ -117,181 +162,196 @@ class KafkaApis(val requestChannel: RequestChannel, // 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 controlledShutdownRequest = request.requestObj.asInstanceOf[ControlledShutdownRequest] + + authorizeClusterAction(request) + 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))) } - private def producerRequestFromOffsetCommit(offsetCommitRequest: OffsetCommitRequest) = { - val msgs = offsetCommitRequest.filterLargeMetadata(config.offsetMetadataMaxSize).map { - case (topicAndPartition, offset) => - new Message( - bytes = OffsetManager.offsetCommitValue(offset), - key = OffsetManager.offsetCommitKey(offsetCommitRequest.groupId, topicAndPartition.topic, topicAndPartition.partition) - ) - }.toSeq - - val producerData = mutable.Map( - TopicAndPartition(OffsetManager.OffsetsTopicName, offsetManager.partitionFor(offsetCommitRequest.groupId)) -> - new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, msgs:_*) - ) - - val request = ProducerRequest( - correlationId = offsetCommitRequest.correlationId, - clientId = offsetCommitRequest.clientId, - requiredAcks = config.offsetCommitRequiredAcks, - ackTimeoutMs = config.offsetCommitTimeoutMs, - data = producerData) - trace("Created producer request %s for offset commit request %s.".format(request, offsetCommitRequest)) - request - } /** - * Handle a produce request or offset commit request (which is really a specialized producer request) + * Handle an offset commit request */ - def handleProducerOrOffsetCommitRequest(request: RequestChannel.Request) { - val (produceRequest, offsetCommitRequestOpt) = - if (request.requestId == RequestKeys.OffsetCommitKey) { - val offsetCommitRequest = request.requestObj.asInstanceOf[OffsetCommitRequest] - (producerRequestFromOffsetCommit(offsetCommitRequest), Some(offsetCommitRequest)) - } else { - (request.requestObj.asInstanceOf[ProducerRequest], None) - } + def handleOffsetCommitRequest(request: RequestChannel.Request) { + val offsetCommitRequest = request.requestObj.asInstanceOf[OffsetCommitRequest] - val sTime = SystemTime.milliseconds - val localProduceResults = appendToLocalLog(produceRequest, offsetCommitRequestOpt.nonEmpty) - debug("Produce to local log in %d ms".format(SystemTime.milliseconds - sTime)) - - val firstErrorCode = localProduceResults.find(_.errorCode != ErrorMapping.NoError).map(_.errorCode).getOrElse(ErrorMapping.NoError) - - val numPartitionsInError = localProduceResults.count(_.error.isDefined) - 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 { + // filter non-exist topics + val invalidRequestsInfo = offsetCommitRequest.requestInfo.filter { case (topicAndPartition, offsetMetadata) => + !metadataCache.contains(topicAndPartition.topic) + } + val filteredRequestInfo = (offsetCommitRequest.requestInfo -- invalidRequestsInfo.keys) - if (firstErrorCode == ErrorMapping.NoError) - offsetCommitRequestOpt.foreach(ocr => offsetManager.putOffsets(ocr.groupId, ocr.requestInfo)) + val (authorizedRequestInfo, unauthorizedRequestInfo) = filteredRequestInfo.partition { + case (topicAndPartition, offsetMetadata) => + authorize(request.session, Read, new Resource(Topic, topicAndPartition.topic)) && + authorize(request.session, Read, new Resource(ConsumerGroup, offsetCommitRequest.groupId)) + } - if (offsetCommitRequestOpt.isDefined) { - val response = offsetCommitRequestOpt.get.responseFor(firstErrorCode, config.offsetMetadataMaxSize) - requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) - } else - requestChannel.noOperation(request.processor, request) + // the callback for sending an offset commit response + def sendResponseCallback(commitStatus: immutable.Map[TopicAndPartition, Short]) { + val mergedCommitStatus = commitStatus ++ unauthorizedRequestInfo.mapValues(_ => ErrorMapping.AuthorizationCode) + + mergedCommitStatus.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))) + } } - } else if (produceRequest.requiredAcks == 1 || - produceRequest.numPartitions <= 0 || - numPartitionsInError == produceRequest.numPartitions) { + val combinedCommitStatus = mergedCommitStatus ++ invalidRequestsInfo.map(_._1 -> ErrorMapping.UnknownTopicOrPartitionCode) + val response = OffsetCommitResponse(combinedCommitStatus, offsetCommitRequest.correlationId) + requestChannel.sendResponse(new RequestChannel.Response(request, new RequestOrResponseSend(request.connectionId, response))) + } - if (firstErrorCode == ErrorMapping.NoError) { - offsetCommitRequestOpt.foreach(ocr => offsetManager.putOffsets(ocr.groupId, ocr.requestInfo) ) + if (offsetCommitRequest.versionId == 0) { + // for version 0 always store offsets to ZK + val responseInfo = authorizedRequestInfo.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]])) + } + } } - val statuses = localProduceResults.map(r => r.key -> ProducerResponseStatus(r.errorCode, r.start)).toMap - val response = offsetCommitRequestOpt.map(_.responseFor(firstErrorCode, config.offsetMetadataMaxSize)) - .getOrElse(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 TopicPartitionRequestKey(topicAndPartition)).toSeq - val statuses = localProduceResults.map(r => - r.key -> DelayedProduceResponseStatus(r.end + 1, ProducerResponseStatus(r.errorCode, r.start))).toMap - val delayedRequest = new DelayedProduce( - producerRequestKeys, - request, - produceRequest.ackTimeoutMs.toLong, - produceRequest, - statuses, - offsetCommitRequestOpt) - - // add the produce request for watch if it's not satisfied, otherwise send the response back - val satisfiedByMe = producerRequestPurgatory.checkAndMaybeWatch(delayedRequest) - if (satisfiedByMe) - producerRequestPurgatory.respond(delayedRequest) - } - - // we do not need the data anymore - produceRequest.emptyData() - } + // 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 + } - 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]]) + // 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 = authorizedRequestInfo.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) } } + private def authorize(session: Session, operation: Operation, resource: Resource): Boolean = + authorizer.map(_.authorize(session, operation, resource)).getOrElse(true) + /** - * Helper method for handling a parsed producer request + * Handle a produce request */ - private def appendToLocalLog(producerRequest: ProducerRequest, isOffsetCommit: Boolean): Iterable[ProduceResult] = { - val partitionAndData: Map[TopicAndPartition, MessageSet] = producerRequest.data - trace("Append [%s] to local log ".format(partitionAndData.toString)) - partitionAndData.map {case (topicAndPartition, messages) => - try { - if (Topic.InternalTopics.contains(topicAndPartition.topic) && - !(isOffsetCommit && topicAndPartition.topic == OffsetManager.OffsetsTopicName)) { - throw new InvalidTopicException("Cannot append to internal topic %s".format(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)) - } + def handleProducerRequest(request: RequestChannel.Request) { + val produceRequest = request.requestObj.asInstanceOf[ProducerRequest] + val numBytesAppended = produceRequest.sizeInBytes - val numAppendedMessages = if (info.firstOffset == -1L || info.lastOffset == -1L) 0 else (info.lastOffset - info.firstOffset + 1) + val (authorizedRequestInfo, unauthorizedRequestInfo) = produceRequest.data.partition { + case (topicAndPartition, _) => authorize(request.session, Write, new Resource(Topic, topicAndPartition.topic)) + } - // 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) + // the callback for sending a produce response + def sendResponseCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { + var errorInResponse = false + val mergedResponseStatus = responseStatus ++ unauthorizedRequestInfo.mapValues(_ => ProducerResponseStatus(ErrorMapping.AuthorizationCode, -1)) + + mergedResponseStatus.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 ite: InvalidTopicException => - warn("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( - producerRequest.correlationId, producerRequest.clientId, topicAndPartition, ite.getMessage)) - new ProduceResult(topicAndPartition, ite) - 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) - } + def produceResponseCallback(delayTimeMs: Int) { + 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, + mergedResponseStatus, + produceRequest.versionId, + delayTimeMs) + requestChannel.sendResponse(new RequestChannel.Response(request, + new RequestOrResponseSend(request.connectionId, + response))) + } + } + + quotaManagers(RequestKeys.ProduceKey).recordAndMaybeThrottle(produceRequest.clientId, + numBytesAppended, + produceResponseCallback) } + + // 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, + authorizedRequestInfo, + 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() } /** @@ -299,67 +359,73 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleFetchRequest(request: RequestChannel.Request) { val fetchRequest = request.requestObj.asInstanceOf[FetchRequest] - val dataRead = replicaManager.readMessageSets(fetchRequest) - - // if the fetch request comes from the follower, - // update its corresponding log end offset - if(fetchRequest.isFromFollower) - recordFollowerLogEndOffsets(fetchRequest.replicaId, dataRead.mapValues(_.offset)) - - // check if this fetch request can be satisfied right away - val bytesReadable = dataRead.values.map(_.data.messages.sizeInBytes).sum - val errorReadingData = dataRead.values.foldLeft(false)((errorIncurred, dataAndOffset) => - errorIncurred || (dataAndOffset.data.error != ErrorMapping.NoError)) - // send the data 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(fetchRequest.maxWait <= 0 || - fetchRequest.numPartitions <= 0 || - bytesReadable >= fetchRequest.minBytes || - errorReadingData) { - debug("Returning fetch response %s for fetch request with correlation id %d to client %s" - .format(dataRead.values.map(_.data.error).mkString(","), fetchRequest.correlationId, fetchRequest.clientId)) - val response = new FetchResponse(fetchRequest.correlationId, dataRead.mapValues(_.data)) - 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 TopicPartitionRequestKey(_)) - val delayedFetch = new DelayedFetch(delayedFetchKeys, request, fetchRequest.maxWait, fetchRequest, - dataRead.mapValues(_.offset)) - - // add the fetch request for watch if it's not satisfied, otherwise send the response back - val satisfiedByMe = fetchRequestPurgatory.checkAndMaybeWatch(delayedFetch) - if (satisfiedByMe) - fetchRequestPurgatory.respond(delayedFetch) + + val (authorizedRequestInfo, unauthorizedRequestInfo) = fetchRequest.requestInfo.partition { + case (topicAndPartition, _) => authorize(request.session, Read, new Resource(Topic, topicAndPartition.topic)) } - } - private def recordFollowerLogEndOffsets(replicaId: Int, offsets: Map[TopicAndPartition, LogOffsetMetadata]) { - debug("Record follower log end offsets: %s ".format(offsets)) - offsets.foreach { - case (topicAndPartition, offset) => - replicaManager.updateReplicaLEOAndPartitionHW(topicAndPartition.topic, - topicAndPartition.partition, replicaId, offset) + val unauthorizedResponseStatus = unauthorizedRequestInfo.mapValues(_ => FetchResponsePartitionData(ErrorMapping.AuthorizationCode, -1, MessageSet.Empty)) + + // the callback for sending a fetch response + def sendResponseCallback(responsePartitionData: Map[TopicAndPartition, FetchResponsePartitionData]) { + val mergedResponseStatus = responsePartitionData ++ unauthorizedResponseStatus - // for producer requests with ack > 1, we need to check - // if they can be unblocked after some follower's log end offsets have moved - replicaManager.unblockDelayedProduceRequests(new TopicPartitionRequestKey(topicAndPartition)) + mergedResponseStatus.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))) + } + // 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) + } + + def fetchResponseCallback(delayTimeMs: Int) { + val response = FetchResponse(fetchRequest.correlationId, responsePartitionData, fetchRequest.versionId, delayTimeMs) + requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(request.connectionId, response))) + } + + // Do not throttle replication traffic + if (fetchRequest.isFromFollower) { + fetchResponseCallback(0) + } else { + quotaManagers(RequestKeys.FetchKey).recordAndMaybeThrottle(fetchRequest.clientId, + FetchResponse.responseSize(responsePartitionData + .groupBy(_._1.topic), + fetchRequest.versionId), + fetchResponseCallback) + } } + + // call the replica manager to fetch messages from the local replica + replicaManager.fetchMessages( + fetchRequest.maxWait.toLong, + fetchRequest.replicaId, + fetchRequest.minBytes, + authorizedRequestInfo, + 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 (authorizedRequestInfo, unauthorizedRequestInfo) = offsetRequest.requestInfo.partition { + case (topicAndPartition, _) => authorize(request.session, Describe, new Resource(Topic, topicAndPartition.topic)) + } + + val unauthorizedResponseStatus = unauthorizedRequestInfo.mapValues(_ => PartitionOffsetsResponse(ErrorMapping.AuthorizationCode, Nil)) + + val responseMap = authorizedRequestInfo.map(elem => { val (topicAndPartition, partitionOffsetRequestInfo) = elem try { // ensure leader exists - val localReplica = if(!offsetRequest.isFromDebuggingClient) + val localReplica = if (!offsetRequest.isFromDebuggingClient) replicaManager.getLeaderReplicaIfLocal(topicAndPartition.topic, topicAndPartition.partition) else replicaManager.getReplicaOrException(topicAndPartition.topic, topicAndPartition.partition) @@ -374,7 +440,7 @@ class KafkaApis(val requestChannel: RequestChannel, val hw = localReplica.highWatermark.messageOffset if (allOffsets.exists(_ > hw)) hw +: allOffsets.dropWhile(_ > hw) - else + else allOffsets } } @@ -383,45 +449,47 @@ 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))) + + val mergedResponseMap = responseMap ++ unauthorizedResponseStatus + val response = OffsetResponse(offsetRequest.correlationId, mergedResponseMap) + 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) + if (segsArray.last.size > 0) offsetTimeArray = new Array[(Long, Long)](segsArray.length + 1) else offsetTimeArray = new Array[(Long, Long)](segsArray.length) for(i <- 0 until segsArray.length) offsetTimeArray(i) = (segsArray(i).baseOffset, segsArray(i).lastModified) - if(segsArray.last.size > 0) + if (segsArray.last.size > 0) offsetTimeArray(segsArray.length) = (log.logEndOffset, SystemTime.milliseconds) var startIndex = -1 @@ -452,28 +520,38 @@ class KafkaApis(val requestChannel: RequestChannel, ret.toSeq.sortBy(- _) } - private def getTopicMetadata(topics: Set[String]): Seq[TopicMetadata] = { - val topicResponses = metadataCache.getTopicMetadata(topics) + 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 == OffsetManager.OffsetsTopicName || config.autoCreateTopicsEnable) { + if (topic == ConsumerCoordinator.OffsetsTopicName || config.autoCreateTopicsEnable) { try { - if (topic == OffsetManager.OffsetsTopicName) { - AdminUtils.createTopic(zkClient, topic, config.offsetsTopicPartitions, config.offsetsTopicReplicationFactor, - offsetManager.offsetsTopicConfig) + 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, config.offsetsTopicReplicationFactor)) + .format(topic, config.offsetsTopicPartitions, offsetsTopicReplicationFactor)) } 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(topic, config.numPartitions, config.defaultReplicationFactor)) } + new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.LeaderNotAvailableCode) } catch { case e: TopicExistsException => // let it go, possibly another broker created this topic + new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.LeaderNotAvailableCode) + case itex: InvalidTopicException => + new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.InvalidTopicCode) } - new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.LeaderNotAvailableCode) } else { new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.UnknownTopicOrPartitionCode) } @@ -484,62 +562,229 @@ class KafkaApis(val requestChannel: RequestChannel, } /** - * Service the topic metadata request API + * Handle a topic metadata request */ def handleTopicMetadataRequest(request: RequestChannel.Request) { val metadataRequest = request.requestObj.asInstanceOf[TopicMetadataRequest] - val topicMetadata = getTopicMetadata(metadataRequest.topics.toSet) + + //if topics is empty -> fetch all topics metadata but filter out the topic response that are not authorized + val topics = if (metadataRequest.topics.isEmpty) { + val topicResponses = metadataCache.getTopicMetadata(metadataRequest.topics.toSet, request.securityProtocol) + topicResponses.map(_.topic).filter(topic => authorize(request.session, Describe, new Resource(Topic, topic))).toSet + } else { + metadataRequest.topics.toSet + } + + //when topics is empty this will be a duplicate authorization check but given this should just be a cache lookup, it should not matter. + var (authorizedTopics, unauthorizedTopics) = topics.partition(topic => authorize(request.session, Describe, new Resource(Topic, topic))) + + if (!authorizedTopics.isEmpty) { + val topicResponses = metadataCache.getTopicMetadata(authorizedTopics, request.securityProtocol) + if (config.autoCreateTopicsEnable && topicResponses.size != authorizedTopics.size) { + val nonExistentTopics: Set[String] = topics -- topicResponses.map(_.topic).toSet + authorizer.foreach { + az => if (!az.authorize(request.session, Create, Resource.ClusterResource)) { + authorizedTopics --= nonExistentTopics + unauthorizedTopics ++= nonExistentTopics + } + } + } + } + + val unauthorizedTopicMetaData = unauthorizedTopics.map(topic => new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.AuthorizationCode)) + + val topicMetadata = getTopicMetadata(authorizedTopics, 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, topicMetadata, metadataRequest.correlationId) - requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) + val response = new TopicMetadataResponse(brokers.map(_.getBrokerEndPoint(request.securityProtocol)), topicMetadata ++ unauthorizedTopicMetaData, 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 status = offsetManager.getOffsets(offsetFetchRequest.groupId, offsetFetchRequest.requestInfo).toMap + val (authorizedTopicPartitions, unauthorizedTopicPartitions) = offsetFetchRequest.requestInfo.partition { topicAndPartition => + authorize(request.session, Describe, new Resource(Topic, topicAndPartition.topic)) && + authorize(request.session, Read, new Resource(ConsumerGroup, offsetFetchRequest.groupId)) + } + + val authorizationError = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, ErrorMapping.AuthorizationCode) + val unauthorizedStatus = unauthorizedTopicPartitions.map(topicAndPartition => (topicAndPartition, authorizationError)).toMap + + val response = if (offsetFetchRequest.versionId == 0) { + // version 0 reads offsets from ZK + val responseInfo = authorizedTopicPartitions.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]]))) + } + }) + + val unauthorizedTopics = unauthorizedTopicPartitions.map( topicAndPartition => + (topicAndPartition, OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata,ErrorMapping.AuthorizationCode))) + OffsetFetchResponse(collection.immutable.Map(responseInfo: _*) ++ unauthorizedTopics, offsetFetchRequest.correlationId) + } else { + // version 1 reads offsets from Kafka; + val offsets = coordinator.handleFetchOffsets(offsetFetchRequest.groupId, authorizedTopicPartitions).toMap - val response = OffsetFetchResponse(status, offsetFetchRequest.correlationId) + // 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 ++ unauthorizedStatus, offsetFetchRequest.correlationId) + } trace("Sending offset fetch response %s for correlation id %d to client %s." .format(response, offsetFetchRequest.correlationId, offsetFetchRequest.clientId)) - requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) + + requestChannel.sendResponse(new RequestChannel.Response(request, new RequestOrResponseSend(request.connectionId, response))) } /* - * Service the consumer metadata API + * Handle a consumer metadata request */ def handleConsumerMetadataRequest(request: RequestChannel.Request) { val consumerMetadataRequest = request.requestObj.asInstanceOf[ConsumerMetadataRequest] - val partition = offsetManager.partitionFor(consumerMetadataRequest.group) + if (!authorize(request.session, Read, new Resource(ConsumerGroup, consumerMetadataRequest.group))) { + val response = ConsumerMetadataResponse(None, ErrorMapping.AuthorizationCode, consumerMetadataRequest.correlationId) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, response))) + } else { + val partition = coordinator.partitionFor(consumerMetadataRequest.group) - // get metadata (and create the topic if necessary) - val offsetsTopicMetadata = getTopicMetadata(Set(OffsetManager.OffsetsTopicName)).head + //get metadata (and create the topic if necessary) + val offsetsTopicMetadata = getTopicMetadata(Set(ConsumerCoordinator.OffsetsTopicName), request.securityProtocol).head - val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, consumerMetadataRequest.correlationId) + val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, consumerMetadataRequest.correlationId) - val response = - offsetsTopicMetadata.partitionsMetadata.find(_.partitionId == partition).map { partitionMetadata => - partitionMetadata.leader.map { leader => - ConsumerMetadataResponse(Some(leader), ErrorMapping.NoError, consumerMetadataRequest.correlationId) + val response = + offsetsTopicMetadata.partitionsMetadata.find(_.partitionId == partition).map { partitionMetadata => + partitionMetadata.leader.map { leader => + ConsumerMetadataResponse(Some(leader), ErrorMapping.NoError, consumerMetadataRequest.correlationId) + }.getOrElse(errorResponse) }.getOrElse(errorResponse) - }.getOrElse(errorResponse) - 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 BoundedByteBufferSend(response))) + trace("Sending consumer metadata %s for correlation id %d to client %s." + .format(response, consumerMetadataRequest.correlationId, consumerMetadataRequest.clientId)) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, response))) + } + } + + def handleJoinGroupRequest(request: RequestChannel.Request) { + import JavaConversions._ + + val joinGroupRequest = request.body.asInstanceOf[JoinGroupRequest] + val respHeader = new ResponseHeader(request.header.correlationId) + + // the callback for sending a join-group response + def sendResponseCallback(partitions: Set[TopicAndPartition], consumerId: String, generationId: Int, errorCode: Short) { + val partitionList = if (errorCode == ErrorMapping.NoError) + partitions.map(tp => new TopicPartition(tp.topic, tp.partition)).toBuffer + else + List.empty.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 Response(request, new ResponseSend(request.connectionId, respHeader, responseBody))) + } + + // ensure that the client is authorized to join the group and read from all subscribed topics + if (!authorize(request.session, Read, new Resource(ConsumerGroup, joinGroupRequest.groupId())) || + joinGroupRequest.topics().exists(topic => !authorize(request.session, Read, new Resource(Topic, topic)))) { + val responseBody = new JoinGroupResponse(ErrorMapping.AuthorizationCode, 0, joinGroupRequest.consumerId(), List.empty[TopicPartition]) + requestChannel.sendResponse(new Response(request, new ResponseSend(request.connectionId, respHeader, responseBody))) + } else { + // let the coordinator to handle join-group + coordinator.handleJoinGroup( + joinGroupRequest.groupId(), + joinGroupRequest.consumerId(), + joinGroupRequest.topics().toSet, + joinGroupRequest.sessionTimeout(), + joinGroupRequest.strategy(), + sendResponseCallback) + } + } + + def handleHeartbeatRequest(request: RequestChannel.Request) { + val heartbeatRequest = request.body.asInstanceOf[HeartbeatRequest] + val respHeader = new ResponseHeader(request.header.correlationId) + + // 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))) + } + + if (!authorize(request.session, Read, new Resource(ConsumerGroup, heartbeatRequest.groupId))) { + val heartbeatResponse = new HeartbeatResponse(ErrorMapping.AuthorizationCode) + requestChannel.sendResponse(new Response(request, new ResponseSend(request.connectionId, respHeader, heartbeatResponse))) + } + else { + // let the coordinator to handle heartbeat + coordinator.handleHeartbeat( + heartbeatRequest.groupId(), + heartbeatRequest.consumerId(), + heartbeatRequest.groupGenerationId(), + sendResponseCallback) + } + } + + /* + * Returns a Map of all quota managers configured. The request Api key is the key for the Map + */ + private def instantiateQuotaManagers(cfg: KafkaConfig): Map[Short, ClientQuotaManager] = { + val producerQuotaManagerCfg = ClientQuotaManagerConfig( + quotaBytesPerSecondDefault = cfg.producerQuotaBytesPerSecondDefault, + quotaBytesPerSecondOverrides = cfg.producerQuotaBytesPerSecondOverrides, + numQuotaSamples = cfg.numQuotaSamples, + quotaWindowSizeSeconds = cfg.quotaWindowSizeSeconds + ) + + val consumerQuotaManagerCfg = ClientQuotaManagerConfig( + quotaBytesPerSecondDefault = cfg.consumerQuotaBytesPerSecondDefault, + quotaBytesPerSecondOverrides = cfg.consumerQuotaBytesPerSecondOverrides, + numQuotaSamples = cfg.numQuotaSamples, + quotaWindowSizeSeconds = cfg.quotaWindowSizeSeconds + ) + + val quotaManagers = Map[Short, ClientQuotaManager]( + RequestKeys.ProduceKey -> + new ClientQuotaManager(producerQuotaManagerCfg, metrics, RequestKeys.nameForKey(RequestKeys.ProduceKey), new org.apache.kafka.common.utils.SystemTime), + RequestKeys.FetchKey -> + new ClientQuotaManager(consumerQuotaManagerCfg, metrics, RequestKeys.nameForKey(RequestKeys.FetchKey), new org.apache.kafka.common.utils.SystemTime) + ) + quotaManagers } def close() { - debug("Shutting down.") - fetchRequestPurgatory.shutdown() - producerRequestPurgatory.shutdown() - debug("Shut down complete.") + quotaManagers.foreach { case (apiKey, quotaManager) => + quotaManager.shutdown() + } + info("Shutdown complete.") + } + + def authorizeClusterAction(request: RequestChannel.Request): Unit = { + if (!authorize(request.session, ClusterAction, Resource.ClusterResource)) + throw new AuthorizationException(s"Request $request is not authorized.") } -} +} 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 165c816a9f4c9..46f4a25073855 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -1,323 +1,954 @@ /** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT 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 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.SSLConfigs +import org.apache.kafka.common.config.ConfigDef.Importance._ +import org.apache.kafka.common.config.ConfigDef.Range._ +import org.apache.kafka.common.config.ConfigDef.Type._ + +import org.apache.kafka.common.config.{ConfigException, AbstractConfig, ConfigDef} +import org.apache.kafka.common.metrics.MetricsReporter +import org.apache.kafka.common.protocol.SecurityProtocol +import org.apache.kafka.common.security.auth.PrincipalBuilder +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 + + /************* Authorizer Configuration ***********/ + val AuthorizerClassName = "" + + /** ********* 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 + val RequestTimeoutMs = 30000 + + /** ********* 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 = RequestTimeoutMs + 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 + + /** ********* Quota Configuration ***********/ + val ProducerQuotaBytesPerSecondDefault = ClientQuotaManagerConfig.QuotaBytesPerSecondDefault + val ConsumerQuotaBytesPerSecondDefault = ClientQuotaManagerConfig.QuotaBytesPerSecondDefault + val ProducerQuotaBytesPerSecondOverrides = ClientQuotaManagerConfig.QuotaBytesPerSecondOverrides + val ConsumerQuotaBytesPerSecondOverrides = ClientQuotaManagerConfig.QuotaBytesPerSecondOverrides + val NumQuotaSamples: Int = ClientQuotaManagerConfig.DefaultNumQuotaSamples + val QuotaWindowSizeSeconds: Int = ClientQuotaManagerConfig.DefaultQuotaWindowSizeSeconds + + val DeleteTopicEnable = false + + val CompressionType = "producer" + + /** ********* Kafka Metrics Configuration ***********/ + val MetricNumSamples = 2 + val MetricSampleWindowMs = 30000 + val MetricReporterClasses = "" + + /** ********* SSL configuration ***********/ + val PrincipalBuilderClass = SSLConfigs.DEFAULT_PRINCIPAL_BUILDER_CLASS + val SSLProtocol = SSLConfigs.DEFAULT_SSL_PROTOCOL + val SSLEnabledProtocols = SSLConfigs.DEFAULT_ENABLED_PROTOCOLS + val SSLKeystoreType = SSLConfigs.DEFAULT_SSL_KEYSTORE_TYPE + val SSLKeystoreLocation = "/tmp/ssl.keystore.jks" + val SSLKeystorePassword = "keystore_password" + val SSLKeyPassword = "key_password" + val SSLTruststoreType = SSLConfigs.DEFAULT_SSL_TRUSTSTORE_TYPE + val SSLTruststoreLocation = SSLConfigs.DEFAULT_TRUSTSTORE_LOCATION + val SSLTruststorePassword = SSLConfigs.DEFAULT_TRUSTSTORE_PASSWORD + val SSLKeyManagerAlgorithm = SSLConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM + val SSLTrustManagerAlgorithm = SSLConfigs.DEFAULT_SSL_TRUSTMANAGER_ALGORITHM + val SSLClientAuthRequired = "required" + val SSLClientAuthRequested = "requested" + val SSLClientAuthNone = "none" + val SSLClientAuth = SSLClientAuthNone -/** - * Configuration settings for the kafka server - */ -class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(props) { +} - def this(originalProps: Properties) { - this(new VerifiableProperties(originalProps)) - props.verify() - } +object KafkaConfig { - private def getLogRetentionTimeMillis(): Long = { - val millisInMinute = 60L * 1000L - val millisInHour = 60L * millisInMinute - - if(props.containsKey("log.retention.ms")){ - props.getIntInRange("log.retention.ms", (1, Int.MaxValue)) - } - else if(props.containsKey("log.retention.minutes")){ - millisInMinute * props.getIntInRange("log.retention.minutes", (1, Int.MaxValue)) - } - else { - millisInHour * props.getIntInRange("log.retention.hours", 24*7, (1, Int.MaxValue)) - } - } - - private def getLogRollTimeMillis(): Long = { - val millisInHour = 60L * 60L * 1000L - - if(props.containsKey("log.roll.ms")){ - props.getIntInRange("log.roll.ms", (1, Int.MaxValue)) - } - else { - millisInHour * props.getIntInRange("log.roll.hours", 24*7, (1, Int.MaxValue)) - } + def main(args: Array[String]) { + System.out.println(configDef.toHtmlTable) } - - /*********** 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", 10, (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) + /** ********* 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" + val RequestTimeoutMsProp = CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG + /************* Authorizer Configuration ***********/ + val AuthorizerClassNameProp = "authorizer.class.name" + /** ********* 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" + /** ********* Quota Configuration ***********/ + val ProducerQuotaBytesPerSecondDefaultProp = "quota.producer.default" + val ConsumerQuotaBytesPerSecondDefaultProp = "quota.consumer.default" + val ProducerQuotaBytesPerSecondOverridesProp = "quota.producer.bytes.per.second.overrides" + val ConsumerQuotaBytesPerSecondOverridesProp = "quota.consumer.bytes.per.second.overrides" + val NumQuotaSamplesProp = "quota.window.num" + val QuotaWindowSizeSecondsProp = "quota.window.size.seconds" + + val DeleteTopicEnableProp = "delete.topic.enable" + val CompressionTypeProp = "compression.type" + + /** ********* Kafka Metrics Configuration ***********/ + val MetricSampleWindowMsProp = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG + val MetricNumSamplesProp: String = CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG + val MetricReporterClassesProp: String = CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG + + /** ********* SSL Configuration ****************/ + val PrincipalBuilderClassProp = SSLConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG + val SSLProtocolProp = SSLConfigs.SSL_PROTOCOL_CONFIG + val SSLProviderProp = SSLConfigs.SSL_PROVIDER_CONFIG + val SSLCipherSuitesProp = SSLConfigs.SSL_CIPHER_SUITES_CONFIG + val SSLEnabledProtocolsProp = SSLConfigs.SSL_ENABLED_PROTOCOLS_CONFIG + val SSLKeystoreTypeProp = SSLConfigs.SSL_KEYSTORE_TYPE_CONFIG + val SSLKeystoreLocationProp = SSLConfigs.SSL_KEYSTORE_LOCATION_CONFIG + val SSLKeystorePasswordProp = SSLConfigs.SSL_KEYSTORE_PASSWORD_CONFIG + val SSLKeyPasswordProp = SSLConfigs.SSL_KEY_PASSWORD_CONFIG + val SSLTruststoreTypeProp = SSLConfigs.SSL_TRUSTSTORE_TYPE_CONFIG + val SSLTruststoreLocationProp = SSLConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG + val SSLTruststorePasswordProp = SSLConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG + val SSLKeyManagerAlgorithmProp = SSLConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG + val SSLTrustManagerAlgorithmProp = SSLConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG + val SSLEndpointIdentificationAlgorithmProp = SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG + val SSLClientAuthProp = SSLConfigs.SSL_CLIENT_AUTH_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" + val RequestTimeoutMsDoc = CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC + /************* Authorizer Configuration ***********/ + val AuthorizerClassNameDoc = "The authorizer class that should be used for authorization" + /** ********* 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 milliseconds 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.9.0.0, 0.9.0.1 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" + /** ********* Quota Configuration ***********/ + val ProducerQuotaBytesPerSecondDefaultDoc = "Any producer distinguished by clientId will get throttled if it produces more bytes than this value per-second" + val ConsumerQuotaBytesPerSecondDefaultDoc = "Any consumer distinguished by clientId/consumer group will get throttled if it fetches more bytes than this value per-second" + val ProducerQuotaBytesPerSecondOverridesDoc = "Comma separated list of clientId:quotaBytesPerSecond to override the default producer quota. " + + "Example: clientIdX=10485760,clientIdY=10485760" + val ConsumerQuotaBytesPerSecondOverridesDoc = "Comma separated list of clientId:quotaBytesPerSecond to override the default consumer quota. " + + "Example: clientIdX=10485760,clientIdY=10485760" + val NumQuotaSamplesDoc = "The number of samples to retain in memory" + val QuotaWindowSizeSecondsDoc = "The time span of each sample" + + 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." + + /** ********* Kafka Metrics Configuration ***********/ + val MetricSampleWindowMsDoc = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC + val MetricNumSamplesDoc = CommonClientConfigs.METRICS_NUM_SAMPLES_DOC + val MetricReporterClassesDoc = CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC + + /** ********* SSL Configuration ****************/ + val PrincipalBuilderClassDoc = SSLConfigs.PRINCIPAL_BUILDER_CLASS_DOC + val SSLProtocolDoc = SSLConfigs.SSL_PROTOCOL_DOC + val SSLProviderDoc = SSLConfigs.SSL_PROVIDER_DOC + val SSLCipherSuitesDoc = SSLConfigs.SSL_CIPHER_SUITES_DOC + val SSLEnabledProtocolsDoc = SSLConfigs.SSL_ENABLED_PROTOCOLS_DOC + val SSLKeystoreTypeDoc = SSLConfigs.SSL_KEYSTORE_TYPE_DOC + val SSLKeystoreLocationDoc = SSLConfigs.SSL_KEYSTORE_LOCATION_DOC + val SSLKeystorePasswordDoc = SSLConfigs.SSL_KEYSTORE_PASSWORD_DOC + val SSLKeyPasswordDoc = SSLConfigs.SSL_KEY_PASSWORD_DOC + val SSLTruststoreTypeDoc = SSLConfigs.SSL_TRUSTSTORE_TYPE_DOC + val SSLTruststorePasswordDoc = SSLConfigs.SSL_TRUSTSTORE_PASSWORD_DOC + val SSLTruststoreLocationDoc = SSLConfigs.SSL_TRUSTSTORE_LOCATION_DOC + val SSLKeyManagerAlgorithmDoc = SSLConfigs.SSL_KEYMANAGER_ALGORITHM_DOC + val SSLTrustManagerAlgorithmDoc = SSLConfigs.SSL_TRUSTMANAGER_ALGORITHM_DOC + val SSLEndpointIdentificationAlgorithmDoc = SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DOC + val SSLClientAuthDoc = SSLConfigs.SSL_CLIENT_AUTH_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) + .define(RequestTimeoutMsProp, INT, Defaults.RequestTimeoutMs, HIGH, RequestTimeoutMsDoc) + + /************* Authorizer Configuration ***********/ + .define(AuthorizerClassNameProp, STRING, Defaults.AuthorizerClassName, LOW, AuthorizerClassNameDoc) + + /** ********* 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) + + /** ********* Kafka Metrics Configuration ***********/ + .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) + + /** ********* Quota configuration ***********/ + .define(ProducerQuotaBytesPerSecondDefaultProp, LONG, Defaults.ProducerQuotaBytesPerSecondDefault, atLeast(1), HIGH, ProducerQuotaBytesPerSecondDefaultDoc) + .define(ConsumerQuotaBytesPerSecondDefaultProp, LONG, Defaults.ConsumerQuotaBytesPerSecondDefault, atLeast(1), HIGH, ConsumerQuotaBytesPerSecondDefaultDoc) + .define(ProducerQuotaBytesPerSecondOverridesProp, STRING, Defaults.ProducerQuotaBytesPerSecondOverrides, HIGH, ProducerQuotaBytesPerSecondOverridesDoc) + .define(ConsumerQuotaBytesPerSecondOverridesProp, STRING, Defaults.ConsumerQuotaBytesPerSecondOverrides, HIGH, ConsumerQuotaBytesPerSecondOverridesDoc) + .define(NumQuotaSamplesProp, INT, Defaults.NumQuotaSamples, atLeast(1), LOW, NumQuotaSamplesDoc) + .define(QuotaWindowSizeSecondsProp, INT, Defaults.QuotaWindowSizeSeconds, atLeast(1), LOW, QuotaWindowSizeSecondsDoc) + + + /** ********* SSL Configuration ****************/ + .define(PrincipalBuilderClassProp, STRING, Defaults.PrincipalBuilderClass, MEDIUM, PrincipalBuilderClassDoc) + .define(SSLProtocolProp, STRING, Defaults.SSLProtocol, MEDIUM, SSLProtocolDoc) + .define(SSLProviderProp, STRING, MEDIUM, SSLProviderDoc, false) + .define(SSLEnabledProtocolsProp, LIST, Defaults.SSLEnabledProtocols, MEDIUM, SSLEnabledProtocolsDoc) + .define(SSLKeystoreTypeProp, STRING, Defaults.SSLKeystoreType, MEDIUM, SSLKeystoreTypeDoc) + .define(SSLKeystoreLocationProp, STRING, Defaults.SSLKeystoreLocation, MEDIUM, SSLKeystoreLocationDoc) + .define(SSLKeystorePasswordProp, STRING, Defaults.SSLKeystorePassword, MEDIUM, SSLKeystorePasswordDoc) + .define(SSLKeyPasswordProp, STRING, Defaults.SSLKeyPassword, MEDIUM, SSLKeyPasswordDoc) + .define(SSLTruststoreTypeProp, STRING, Defaults.SSLTruststoreType, MEDIUM, SSLTruststoreTypeDoc) + .define(SSLTruststoreLocationProp, STRING, Defaults.SSLTruststoreLocation, MEDIUM, SSLTruststoreLocationDoc) + .define(SSLTruststorePasswordProp, STRING, Defaults.SSLTruststorePassword, MEDIUM, SSLTruststorePasswordDoc) + .define(SSLKeyManagerAlgorithmProp, STRING, Defaults.SSLKeyManagerAlgorithm, MEDIUM, SSLKeyManagerAlgorithmDoc) + .define(SSLTrustManagerAlgorithmProp, STRING, Defaults.SSLTrustManagerAlgorithm, MEDIUM, SSLTrustManagerAlgorithmDoc) + .define(SSLClientAuthProp, STRING, Defaults.SSLClientAuth, in(Defaults.SSLClientAuthRequired, Defaults.SSLClientAuthRequested, Defaults.SSLClientAuthNone), MEDIUM, SSLClientAuthDoc) - /* 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)) - - /* the maximum number of connections we allow from each ip address */ - val maxConnectionsPerIp: Int = props.getIntInRange("max.connections.per.ip", Int.MaxValue, (1, Int.MaxValue)) - - /* per-ip or hostname overrides to the default maximum number of connections */ - val maxConnectionsPerIpOverrides = props.getMap("max.connections.per.ip.overrides").map(entry => (entry._1, entry._2.toInt)) - - /* idle connections timeout: the server socket processor threads close the connections that idle more than this */ - val connectionsMaxIdleMs = props.getLong("connections.max.idle.ms", 10*60*1000L) - - /*********** Log Configuration ***********/ + } - /* the default number of log partitions per topic */ - val numPartitions = props.getIntInRange("num.partitions", 1, (1, Int.MaxValue)) + def configNames() = { + import scala.collection.JavaConversions._ + configDef.names().toList.sorted + } - /* 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) + /** + * Check that property names are valid + */ + def validateNames(props: Properties) { + import scala.collection.JavaConversions._ + val names = configDef.names() + for (name <- props.keys) + require(names.contains(name), "Unknown configuration \"%s\".".format(name)) + } - /* the maximum size of a single log file */ - val logSegmentBytes = props.getIntInRange("log.segment.bytes", 1*1024*1024*1024, (Message.MinHeaderSize, Int.MaxValue)) + def fromProps(props: Properties): KafkaConfig = { + KafkaConfig(props) + } - /* the maximum time before a new log segment is rolled out */ - val logRollTimeMillis = getLogRollTimeMillis + def fromProps(defaults: Properties, overrides: Properties): KafkaConfig = { + val props = new Properties() + props.putAll(defaults) + props.putAll(overrides) + fromProps(props) + } +} - /* the number of hours to keep a log file before deleting it */ +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) + val requestTimeoutMs = getInt(KafkaConfig.RequestTimeoutMsProp) + + /************* Authorizer Configuration ***********/ + val authorizerClassName: String = getString(KafkaConfig.AuthorizerClassNameProp) + + /** ********* 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 logRetentionTimeMillis = getLogRetentionTimeMillis + val minInSyncReplicas = getInt(KafkaConfig.MinInSyncReplicasProp) + val logPreAllocateEnable: java.lang.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: java.lang.Boolean = 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]) + + /** ********* SSL Configuration **************/ + val principalBuilderClass = getString(KafkaConfig.PrincipalBuilderClassProp) + val sslProtocol = getString(KafkaConfig.SSLProtocolProp) + val sslProvider = getString(KafkaConfig.SSLProviderProp) + val sslEnabledProtocols = getList(KafkaConfig.SSLEnabledProtocolsProp) + val sslKeystoreType = getString(KafkaConfig.SSLKeystoreTypeProp) + val sslKeystoreLocation = getString(KafkaConfig.SSLKeystoreLocationProp) + val sslKeystorePassword = getString(KafkaConfig.SSLKeystorePasswordProp) + val sslKeyPassword = getString(KafkaConfig.SSLKeyPasswordProp) + val sslTruststoreType = getString(KafkaConfig.SSLTruststoreTypeProp) + val sslTruststoreLocation = getString(KafkaConfig.SSLTruststoreLocationProp) + val sslTruststorePassword = getString(KafkaConfig.SSLTruststorePasswordProp) + val sslKeyManagerAlgorithm = getString(KafkaConfig.SSLKeyManagerAlgorithmProp) + val sslTrustManagerAlgorithm = getString(KafkaConfig.SSLTrustManagerAlgorithmProp) + val sslClientAuth = getString(KafkaConfig.SSLClientAuthProp) + + /** ********* Quota Configuration **************/ + val producerQuotaBytesPerSecondDefault = getLong(KafkaConfig.ProducerQuotaBytesPerSecondDefaultProp) + val consumerQuotaBytesPerSecondDefault = getLong(KafkaConfig.ConsumerQuotaBytesPerSecondDefaultProp) + val producerQuotaBytesPerSecondOverrides = getString(KafkaConfig.ProducerQuotaBytesPerSecondOverridesProp) + val consumerQuotaBytesPerSecondOverrides = getString(KafkaConfig.ConsumerQuotaBytesPerSecondOverridesProp) + val numQuotaSamples = getInt(KafkaConfig.NumQuotaSamplesProp) + val quotaWindowSizeSeconds = getInt(KafkaConfig.QuotaWindowSizeSecondsProp) + + val deleteTopicEnable = getBoolean(KafkaConfig.DeleteTopicEnableProp) + val compressionType = getString(KafkaConfig.CompressionTypeProp) + + val listeners = getListeners + val advertisedListeners = getAdvertisedListeners + + private def getLogRetentionTimeMillis: Long = { + val millisInMinute = 60L * 1000L + val millisInHour = 60L * millisInMinute - /* 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 "compact" */ - 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", 15*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)) - - /* the number of threads per data directory to be used for log recovery at startup and flushing at shutdown */ - val numRecoveryThreadsPerDataDir = props.getIntInRange("num.recovery.threads.per.data.dir", 1, (1, 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. Its value should be at least replica.fetch.wait.max.ms. */ - val replicaSocketTimeoutMs = props.getInt("replica.socket.timeout.ms", ConsumerConfig.SocketTimeout) - require(replicaFetchWaitMaxMs <= replicaSocketTimeoutMs, "replica.socket.timeout.ms should always be at least replica.fetch.wait.max.ms" + - " to prevent unnecessary socket timeouts") - - /* 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", 1000) - - /* the purge interval (in number of requests) of the producer request purgatory */ - val producerPurgatoryPurgeIntervalRequests = props.getInt("producer.purgatory.purge.interval.requests", 1000) - - /* 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) - - /* 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 uncleanLeaderElectionEnable = props.getBoolean("unclean.leader.election.enable", true) + 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 + }) - /*********** Controlled shutdown configuration ***********/ + if (millis < 0) return -1 + millis + } - /** 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) + 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)) + } + } - /** 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) + private def validateUniquePortAndProtocol(listeners: String) { - /* enable controlled shutdown of the server */ - val controlledShutdownEnable = props.getBoolean("controlled.shutdown.enable", default = true) + 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)) + } + // filter port 0 for unit tests + val endpointsWithoutZeroPort = endpoints.map(ep => ep.port).filter(_ != 0) + val distinctPorts = endpointsWithoutZeroPort.distinct + val distinctProtocols = endpoints.map(ep => ep.protocolType).distinct - /*********** Offset management configuration ***********/ + require(distinctPorts.size == endpointsWithoutZeroPort.size, "Each listener must have a different port") + require(distinctProtocols.size == endpoints.size, "Each listener must have a different protocol") + } - /* the maximum size for a metadata entry associated with an offset commit */ - val offsetMetadataMaxSize = props.getInt("offset.metadata.max.bytes", OffsetManagerConfig.DefaultMaxMetadataSize) + // 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 { + CoreUtils.listenerListToEndPoints("PLAINTEXT://" + hostName + ":" + port) + } + } - /** Batch size for reading from the offsets segments when loading offsets into the cache. */ - val offsetsLoadBufferSize = props.getIntInRange("offsets.load.buffer.size", - OffsetManagerConfig.DefaultLoadBufferSize, (1, Integer.MAX_VALUE)) + // 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 replication factor for the offset commit topic (set higher to ensure availability). */ - val offsetsTopicReplicationFactor: Short = props.getShortInRange("offsets.topic.replication.factor", - OffsetManagerConfig.DefaultOffsetsTopicReplicationFactor, (1, Short.MaxValue)) + private def getMetricClasses(metricClasses: java.util.List[String]): java.util.List[MetricsReporter] = { - /** The number of partitions for the offset commit topic (should not change after deployment). */ - val offsetsTopicPartitions: Int = props.getIntInRange("offsets.topic.num.partitions", - OffsetManagerConfig.DefaultOffsetsTopicNumPartitions, (1, Integer.MAX_VALUE)) + val reporterList = new util.ArrayList[MetricsReporter](); + val iterator = metricClasses.iterator() - /** The offsets topic segment bytes should be kept relatively small in order to facilitate faster log compaction and cache loads */ - val offsetsTopicSegmentBytes: Int = props.getIntInRange("offsets.topic.segment.bytes", - OffsetManagerConfig.DefaultOffsetsTopicSegmentBytes, (1, Integer.MAX_VALUE)) + while (iterator.hasNext) { + val reporterName = iterator.next() + if (!reporterName.isEmpty) { + val reporter: MetricsReporter = CoreUtils.createObject[MetricsReporter](reporterName) + reporter.configure(originals) + reporterList.add(reporter) + } + } - /** Compression codec for the offsets topic - compression may be used to achieve "atomic" commits. */ - val offsetsTopicCompressionCodec = props.getCompressionCodec("offsets.topic.compression.codec", - OffsetManagerConfig.DefaultOffsetsTopicCompressionCodec) + reporterList - /** Offsets older than this retention period will be discarded. */ - val offsetsRetentionMinutes: Int = props.getIntInRange("offsets.retention.minutes", 24*60, (1, Integer.MAX_VALUE)) + } - /** Frequency at which to check for stale offsets. */ - val offsetsRetentionCheckIntervalMs: Long = props.getLongInRange("offsets.retention.check.interval.ms", - OffsetManagerConfig.DefaultOffsetsRetentionCheckIntervalMs, (1, Long.MaxValue)) - /* 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 offsetCommitTimeoutMs = props.getIntInRange("offsets.commit.timeout.ms", - OffsetManagerConfig.DefaultOffsetCommitTimeoutMs, (1, Integer.MAX_VALUE)) + private def getPrincipalBuilderClass(principalBuilderClass: String): PrincipalBuilder = { + CoreUtils.createObject[PrincipalBuilder](principalBuilderClass) + } - /** The required acks before the commit can be accepted. In general, the default (-1) should not be overridden. */ - val offsetCommitRequiredAcks = props.getShortInRange("offsets.commit.required.acks", - OffsetManagerConfig.DefaultOffsetCommitRequiredAcks, (-1, offsetsTopicReplicationFactor)) + 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(",")) + } - /* Enables delete topic. Delete topic through the admin tool will have no effect if this config is turned off */ - val deleteTopicEnable = props.getBoolean("delete.topic.enable", false) + def channelConfigs: java.util.Map[String, Object] = { + val channelConfigs = new java.util.HashMap[String, Object]() + import kafka.server.KafkaConfig._ + channelConfigs.put(PrincipalBuilderClassProp, Class.forName(principalBuilderClass)) + channelConfigs.put(SSLProtocolProp, sslProtocol) + channelConfigs.put(SSLEnabledProtocolsProp, sslEnabledProtocols) + channelConfigs.put(SSLKeystoreTypeProp, sslKeystoreType) + channelConfigs.put(SSLKeystoreLocationProp, sslKeystoreLocation) + channelConfigs.put(SSLKeystorePasswordProp, sslKeystorePassword) + channelConfigs.put(SSLKeyPasswordProp, sslKeyPassword) + channelConfigs.put(SSLTruststoreTypeProp, sslTruststoreType) + channelConfigs.put(SSLTruststoreLocationProp, sslTruststoreLocation) + channelConfigs.put(SSLTruststorePasswordProp, sslTruststorePassword) + channelConfigs.put(SSLKeyManagerAlgorithmProp, sslKeyManagerAlgorithm) + channelConfigs.put(SSLTrustManagerAlgorithmProp, sslTrustManagerAlgorithm) + channelConfigs.put(SSLClientAuthProp, sslClientAuth) + channelConfigs + } } diff --git a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala index 4acdd70fe9c1e..16760d4f6d3e6 100644 --- a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala +++ b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala @@ -17,9 +17,11 @@ 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 org.I0Itec.zkclient.{IZkStateListener, ZkClient, ZkConnection} import java.net.InetAddress @@ -31,36 +33,36 @@ 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, - private val zkSessionTimeoutMs: Int, - private val zkClient: ZkClient) extends Logging { +class KafkaHealthcheck(private val brokerId: Int, + private val advertisedEndpoints: Map[SecurityProtocol, EndPoint], + private val zkClient: ZkClient, + private val zkConnection: ZkConnection) extends Logging { val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + brokerId val sessionExpireListener = new SessionExpireListener - + def startup() { zkClient.subscribeStateChanges(sessionExpireListener) register() } - def shutdown() { - zkClient.unsubscribeStateChanges(sessionExpireListener) - ZkUtils.deregisterBrokerInZk(zkClient, brokerId) - } - /** * 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, zkConnection, brokerId, plaintextEndpoint.host, plaintextEndpoint.port, updatedEndpoints, jmxPort) } /** @@ -69,9 +71,7 @@ class KafkaHealthcheck(private val brokerId: Int, */ class SessionExpireListener() extends IZkStateListener { @throws(classOf[Exception]) - def handleStateChanged(state: KeeperState) { - // do nothing, since zkclient will do reconnect for us. - } + def handleStateChanged(state: KeeperState) {} /** * Called after the zookeeper session has expired and a new session has been created. You would have to re-create @@ -87,6 +87,10 @@ 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) + } } } 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 00bcc06716fbb..a1558afed20bc --- a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala +++ b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala @@ -22,6 +22,7 @@ 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. @@ -93,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 bytesRejectedRate = newMeter(name + "BytesRejectedPerSec", "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 3e9e91f2b456b..e2c8f48f9b729 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -17,46 +17,122 @@ package kafka.server +import java.net.{SocketTimeoutException} +import java.util + import kafka.admin._ +import kafka.api.{KAFKA_090, ApiVersion} 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 java.net.BindException -import org.I0Itec.zkclient.ZkClient +import java.io.{IOException, File} + +import kafka.security.auth.Authorizer +import kafka.utils._ +import org.apache.kafka.clients.{ManualMetadataUpdater, ClientRequest, NetworkClient} +import org.apache.kafka.common.Node +import org.apache.kafka.common.metrics._ +import org.apache.kafka.common.network.{Selectable, ChannelBuilders, NetworkReceive, Selector} +import org.apache.kafka.common.protocol.{Errors, ApiKeys, SecurityProtocol} +import org.apache.kafka.common.metrics.{JmxReporter, Metrics} +import org.apache.kafka.common.requests.{ControlledShutdownResponse, ControlledShutdownRequest, RequestSend} +import org.apache.kafka.common.security.ssl.SSLFactory +import org.apache.kafka.common.utils.AppInfoParser + +import scala.collection.mutable +import scala.collection.JavaConverters._ +import org.I0Itec.zkclient.{ZkClient, ZkConnection} import kafka.controller.{ControllerStats, KafkaController} -import kafka.cluster.Broker -import kafka.api.{ControlledShutdownResponse, ControlledShutdownRequest} -import kafka.common.ErrorMapping -import kafka.network.{Receive, BlockingChannel, SocketServer} +import kafka.cluster.{EndPoint, Broker} +import kafka.common.{ErrorMapping, InconsistentBrokerIdException, GenerateBrokerIdException} +import kafka.network.{BlockingChannel, SocketServer} import kafka.metrics.KafkaMetricsGroup import com.yammer.metrics.core.Gauge +import kafka.coordinator.{ConsumerCoordinator} + +object KafkaServer { + // 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[kafka] def copyKafkaConfigToLog(kafkaConfig: KafkaConfig): java.util.Map[String, Object] = { + val logProps = new util.HashMap[String, Object]() + logProps.put(LogConfig.SegmentBytesProp, kafkaConfig.logSegmentBytes) + logProps.put(LogConfig.SegmentMsProp, kafkaConfig.logRollTimeMillis) + logProps.put(LogConfig.SegmentJitterMsProp, kafkaConfig.logRollTimeJitterMillis) + logProps.put(LogConfig.SegmentIndexBytesProp, kafkaConfig.logIndexSizeMaxBytes) + logProps.put(LogConfig.FlushMessagesProp, kafkaConfig.logFlushIntervalMessages) + logProps.put(LogConfig.FlushMsProp, kafkaConfig.logFlushIntervalMs) + logProps.put(LogConfig.RetentionBytesProp, kafkaConfig.logRetentionBytes) + logProps.put(LogConfig.RetentionMsProp, kafkaConfig.logRetentionTimeMillis: java.lang.Long) + logProps.put(LogConfig.MaxMessageBytesProp, kafkaConfig.messageMaxBytes) + logProps.put(LogConfig.IndexIntervalBytesProp, kafkaConfig.logIndexIntervalBytes) + logProps.put(LogConfig.DeleteRetentionMsProp, kafkaConfig.logCleanerDeleteRetentionMs) + logProps.put(LogConfig.FileDeleteDelayMsProp, kafkaConfig.logDeleteDelayMs) + logProps.put(LogConfig.MinCleanableDirtyRatioProp, kafkaConfig.logCleanerMinCleanRatio) + logProps.put(LogConfig.CleanupPolicyProp, kafkaConfig.logCleanupPolicy) + logProps.put(LogConfig.MinInSyncReplicasProp, kafkaConfig.minInSyncReplicas) + logProps.put(LogConfig.CompressionTypeProp, kafkaConfig.compressionType) + logProps.put(LogConfig.UncleanLeaderElectionEnableProp, kafkaConfig.uncleanLeaderElectionEnable) + logProps.put(LogConfig.PreAllocateEnableProp, kafkaConfig.logPreAllocateEnable) + logProps + } +} + + /** * 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 with KafkaMetricsGroup { - this.logIdent = "[Kafka Server " + config.brokerId + "], " - private var isShuttingDown = new AtomicBoolean(false) +class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePrefix: Option[String] = None) 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) + + private val jmxPrefix: String = "kafka.server" + private val reporters: java.util.List[MetricsReporter] = config.metricReporterClasses + reporters.add(new JmxReporter(jmxPrefix)) + + // This exists because the Metrics package from clients has its own Time implementation. + // SocketServer/Quotas (which uses client libraries) have to use the client Time objects without having to convert all of Kafka to use them + // Eventually, we want to merge the Time objects in core and clients + private implicit val kafkaMetricsTime: org.apache.kafka.common.utils.Time = new org.apache.kafka.common.utils.SystemTime() + var metrics: Metrics = null + + private val metricConfig: MetricConfig = new MetricConfig() + .samples(config.metricNumSamples) + .timeWindow(config.metricSampleWindowMs, TimeUnit.MILLISECONDS) + val brokerState: BrokerState = new BrokerState - val correlationId: AtomicInteger = new AtomicInteger(0) + + var apis: KafkaApis = null var socketServer: SocketServer = null var requestHandlerPool: KafkaRequestHandlerPool = null + var logManager: LogManager = null - var offsetManager: OffsetManager = null - var kafkaHealthcheck: KafkaHealthcheck = null - var topicConfigManager: TopicConfigManager = null + var replicaManager: ReplicaManager = null - var apis: KafkaApis = null + + var dynamicConfigHandlers: Map[String, ConfigHandler] = null + var dynamicConfigManager: DynamicConfigManager = 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 + var zkConnection: ZkConnection = 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", @@ -72,71 +148,104 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg def startup() { try { info("starting") - brokerState.newState(Starting) - isShuttingDown = new AtomicBoolean(false) - shutdownLatch = new CountDownLatch(1) - - /* start scheduler */ - kafkaScheduler.startup() - - /* setup zookeeper */ - zkClient = initZk() - - /* start log manager */ - logManager = createLogManager(zkClient, brokerState) - logManager.startup() - - socketServer = new SocketServer(config.brokerId, - config.hostName, - config.port, - config.numNetworkThreads, - config.queuedMaxRequests, - config.socketSendBufferBytes, - config.socketReceiveBufferBytes, - config.socketRequestMaxBytes, - config.maxConnectionsPerIp, - config.connectionsMaxIdleMs) - socketServer.startup() - - replicaManager = new ReplicaManager(config, time, zkClient, kafkaScheduler, logManager, isShuttingDown) - - /* start offset manager */ - offsetManager = createOffsetManager() - - kafkaController = new KafkaController(config, zkClient, brokerState) - - /* start processing requests */ - apis = new KafkaApis(socketServer.requestChannel, replicaManager, offsetManager, zkClient, config.brokerId, config, kafkaController) - requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads) - brokerState.newState(RunningAsBroker) - - 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") + + 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) { + metrics = new Metrics(metricConfig, reporters, kafkaMetricsTime) + + brokerState.newState(Starting) + + /* start scheduler */ + kafkaScheduler.startup() + + /* setup zookeeper */ + val (client, connection) = initZk() + zkClient = client + zkConnection = connection + + /* start log manager */ + logManager = createLogManager(zkClient, brokerState) + logManager.startup() + + /* generate brokerId */ + config.brokerId = getBrokerId + this.logIdent = "[Kafka Server " + config.brokerId + "], " + + socketServer = new SocketServer(config, metrics, kafkaMetricsTime) + socketServer.startup() + + /* start replica manager */ + replicaManager = new ReplicaManager(config, metrics, time, kafkaMetricsTime, zkClient, kafkaScheduler, logManager, + isShuttingDown) + replicaManager.startup() + + /* start kafka controller */ + kafkaController = new KafkaController(config, zkClient, zkConnection, brokerState, kafkaMetricsTime, metrics, threadNamePrefix) + kafkaController.startup() + + /* start kafka coordinator */ + consumerCoordinator = ConsumerCoordinator.create(config, zkClient, replicaManager, kafkaScheduler) + consumerCoordinator.startup() + + /* Get the authorizer and initialize it if one is specified.*/ + val authorizer: Option[Authorizer] = if (config.authorizerClassName != null && !config.authorizerClassName.isEmpty) { + val authZ: Authorizer = CoreUtils.createObject(config.authorizerClassName) + authZ.configure(config.originals()) + Option(authZ) + } else { + None + } + + /* start processing requests */ + apis = new KafkaApis(socketServer.requestChannel, replicaManager, consumerCoordinator, + kafkaController, zkClient, config.brokerId, config, metadataCache, metrics, authorizer) + requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads) + brokerState.newState(RunningAsBroker) + + Mx4jLoader.maybeLoad() + + /* start dynamic config manager */ + dynamicConfigHandlers = Map[String, ConfigHandler](ConfigType.Topic -> new TopicConfigHandler(logManager), + ConfigType.Client -> new ClientIdConfigHandler) + dynamicConfigManager = new DynamicConfigManager(zkClient, dynamicConfigHandlers) + dynamicConfigManager.startup() + + /* tell everyone we are alive */ + val listeners = config.advertisedListeners.map {case(protocol, endpoint) => + if (endpoint.port == 0) + (protocol, EndPoint(endpoint.host, socketServer.boundPort(protocol), endpoint.protocolType)) + else + (protocol, endpoint) + } + kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, listeners, zkClient, zkConnection) + kafkaHealthcheck.startup() + + /* register broker metrics */ + registerStats() + + shutdownLatch = new CountDownLatch(1) + startupComplete.set(true) + isStartingUp.set(false) + AppInfoParser.registerAppInfo(jmxPrefix, config.brokerId.toString) + 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 = { + private def initZk(): (ZkClient, ZkConnection) = { info("Connecting to zookeeper on " + config.zkConnect) val chroot = { @@ -148,15 +257,15 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg if (chroot.length > 1) { val zkConnForChrootCreation = config.zkConnect.substring(0, config.zkConnect.indexOf("/")) - val zkClientForChrootCreation = new ZkClient(zkConnForChrootCreation, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, ZKStringSerializer) + val zkClientForChrootCreation = ZkUtils.createZkClient(zkConnForChrootCreation, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs) ZkUtils.makeSurePersistentPathExists(zkClientForChrootCreation, chroot) info("Created zookeeper path " + chroot) zkClientForChrootCreation.close() } - val zkClient = new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, ZKStringSerializer) + val (zkClient, zkConnection) = ZkUtils.createZkClientAndConnection(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs) ZkUtils.setupCommonPaths(zkClient) - zkClient + (zkClient, zkConnection) } @@ -173,18 +282,128 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg * Performs controlled shutdown */ private def controlledShutdown() { - if (startupComplete.get() && config.controlledShutdownEnable) { - // We request the controller to do a controlled shutdown. On failure, we backoff for a configured period - // of time and try again for a configured number of retries. If all the attempt fails, we simply force - // the shutdown. - var remainingRetries = config.controlledShutdownMaxRetries - info("Starting controlled shutdown") - var channel : BlockingChannel = null - var prevController : Broker = null - var shutdownSuceeded : Boolean = false + + def node(broker: Broker): Node = { + val brokerEndPoint = broker.getBrokerEndPoint(config.interBrokerSecurityProtocol) + new Node(brokerEndPoint.id, brokerEndPoint.host, brokerEndPoint.port) + } + + val socketTimeoutMs = config.controllerSocketTimeoutMs + + def socketTimeoutException: Throwable = + new SocketTimeoutException(s"Did not receive response within $socketTimeoutMs") + + def networkClientControlledShutdown(retries: Int): Boolean = { + val metadataUpdater = new ManualMetadataUpdater() + val networkClient = { + val selector = new Selector( + NetworkReceive.UNLIMITED, + config.connectionsMaxIdleMs, + metrics, + kafkaMetricsTime, + "kafka-server-controlled-shutdown", + Map.empty.asJava, + false, + ChannelBuilders.create(config.interBrokerSecurityProtocol, SSLFactory.Mode.CLIENT, config.channelConfigs) + ) + new NetworkClient( + selector, + metadataUpdater, + config.brokerId.toString, + 1, + 0, + Selectable.USE_DEFAULT_BUFFER_SIZE, + Selectable.USE_DEFAULT_BUFFER_SIZE, + config.requestTimeoutMs) + } + + var shutdownSucceeded: Boolean = false + try { - brokerState.newState(PendingControlledShutdown) - while (!shutdownSuceeded && remainingRetries > 0) { + + var remainingRetries = retries + var prevController: Broker = null + var ioException = false + + while (!shutdownSucceeded && remainingRetries > 0) { + remainingRetries = remainingRetries - 1 + + import NetworkClientBlockingOps._ + + // 1. Find the controller and establish a connection to it. + + // Get the current controller info. This is to ensure we use the most recent info to issue the + // controlled shutdown request + val controllerId = ZkUtils.getController(zkClient) + ZkUtils.getBrokerInfo(zkClient, controllerId) match { + case Some(broker) => + // if this is the first attempt, if the controller has changed or if an exception was thrown in a previous + // attempt, connect to the most recent controller + if (ioException || broker != prevController) { + + ioException = false + + if (prevController != null) + networkClient.close(node(prevController).idString) + + prevController = broker + metadataUpdater.setNodes(Seq(node(prevController)).asJava) + } + case None => //ignore and try again + } + + // 2. issue a controlled shutdown to the controller + if (prevController != null) { + try { + + if (!networkClient.blockingReady(node(prevController), socketTimeoutMs)) + throw socketTimeoutException + + // send the controlled shutdown request + val requestHeader = networkClient.nextRequestHeader(ApiKeys.CONTROLLED_SHUTDOWN_KEY) + val send = new RequestSend(node(prevController).idString, requestHeader, + new ControlledShutdownRequest(config.brokerId).toStruct) + val request = new ClientRequest(kafkaMetricsTime.milliseconds(), true, send, null) + val clientResponse = networkClient.blockingSendAndReceive(request, socketTimeoutMs).getOrElse { + throw socketTimeoutException + } + + val shutdownResponse = new ControlledShutdownResponse(clientResponse.responseBody) + if (shutdownResponse.errorCode == Errors.NONE.code && shutdownResponse.partitionsRemaining.isEmpty) { + shutdownSucceeded = true + info("Controlled shutdown succeeded") + } + else { + info("Remaining partitions to move: %s".format(shutdownResponse.partitionsRemaining.asScala.mkString(","))) + info("Error code from controller: %d".format(shutdownResponse.errorCode)) + } + } + catch { + case ioe: IOException => + ioException = true + 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 (!shutdownSucceeded) { + Thread.sleep(config.controlledShutdownRetryBackoffMs) + warn("Retrying controlled shutdown after the previous attempt failed...") + } + } + } + finally + networkClient.close() + + shutdownSucceeded + } + + def blockingChannelControlledShutdown(retries: Int): Boolean = { + var remainingRetries = retries + var channel: BlockingChannel = null + var prevController: Broker = null + var shutdownSucceeded: Boolean = false + try { + while (!shutdownSucceeded && remainingRetries > 0) { remainingRetries = remainingRetries - 1 // 1. Find the controller and establish a connection to it. @@ -197,34 +416,33 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg if (channel == null || prevController == null || !prevController.equals(broker)) { // if this is the first attempt or if the controller has changed, create a channel to the most recent // controller - if (channel != null) { + 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) channel.connect() prevController = broker } - case None=> - //ignore and try again + case None => //ignore and try again } // 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) + val request = new kafka.api.ControlledShutdownRequest(0, correlationId.getAndIncrement, None, config.brokerId) channel.send(request) response = channel.receive() - - val shutdownResponse = ControlledShutdownResponse.readFrom(response.buffer) + val shutdownResponse = kafka.api.ControlledShutdownResponse.readFrom(response.payload()) if (shutdownResponse.errorCode == ErrorMapping.NoError && shutdownResponse.partitionsRemaining != null && - shutdownResponse.partitionsRemaining.size == 0) { - shutdownSuceeded = true + shutdownResponse.partitionsRemaining.size == 0) { + shutdownSucceeded = true info ("Controlled shutdown succeeded") } else { @@ -236,10 +454,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...") } @@ -251,9 +470,27 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg channel = null } } - if (!shutdownSuceeded) { + shutdownSucceeded + } + + if (startupComplete.get() && config.controlledShutdownEnable) { + // We request the controller to do a controlled shutdown. On failure, we backoff for a configured period + // of time and try again for a configured number of retries. If all the attempt fails, we simply force + // the shutdown. + info("Starting controlled shutdown") + + brokerState.newState(PendingControlledShutdown) + + val shutdownSucceeded = + // Before 0.9.0.0, `ControlledShutdownRequest` did not contain `client_id` and it's a mandatory field in + // `RequestHeader`, which is used by `NetworkClient` + if (config.interBrokerProtocolVersion.onOrAfter(KAFKA_090)) + networkClientControlledShutdown(config.controlledShutdownMaxRetries.intValue) + else blockingChannelControlledShutdown(config.controlledShutdownMaxRetries.intValue) + + if (!shutdownSucceeded) warn("Proceeding to do an unclean shutdown as all the controlled shutdown attempts failed") - } + } } @@ -264,39 +501,47 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg def shutdown() { 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) { - Utils.swallow(controlledShutdown()) + if (canShutdown && shutdownLatch.getCount > 0) { + CoreUtils.swallow(controlledShutdown()) brokerState.newState(BrokerShuttingDown) - if(kafkaHealthcheck != null) - Utils.swallow(kafkaHealthcheck.shutdown()) if(socketServer != null) - Utils.swallow(socketServer.shutdown()) + CoreUtils.swallow(socketServer.shutdown()) if(requestHandlerPool != null) - Utils.swallow(requestHandlerPool.shutdown()) - if(offsetManager != null) - offsetManager.shutdown() - Utils.swallow(kafkaScheduler.shutdown()) + CoreUtils.swallow(requestHandlerPool.shutdown()) + CoreUtils.swallow(kafkaScheduler.shutdown()) if(apis != null) - Utils.swallow(apis.close()) + CoreUtils.swallow(apis.close()) if(replicaManager != null) - Utils.swallow(replicaManager.shutdown()) + CoreUtils.swallow(replicaManager.shutdown()) if(logManager != null) - Utils.swallow(logManager.shutdown()) + CoreUtils.swallow(logManager.shutdown()) + if(consumerCoordinator != null) + CoreUtils.swallow(consumerCoordinator.shutdown()) if(kafkaController != null) - Utils.swallow(kafkaController.shutdown()) + CoreUtils.swallow(kafkaController.shutdown()) if(zkClient != null) - Utils.swallow(zkClient.close()) + CoreUtils.swallow(zkClient.close()) + if (metrics != null) + CoreUtils.swallow(metrics.close()) brokerState.newState(NotRunning) - shutdownLatch.countDown() + startupComplete.set(false) + isShuttingDown.set(false) + AppInfoParser.unregisterAppInfo(jmxPrefix, config.brokerId.toString) + shutdownLatch.countDown() info("shut down completed") } } catch { case e: Throwable => fatal("Fatal error during KafkaServer shutdown.", e) + isShuttingDown.set(false) throw e } } @@ -307,22 +552,13 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg def awaitShutdown(): Unit = shutdownLatch.await() def getLogManager(): LogManager = logManager - + + def boundPort(protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): Int = socketServer.boundPort(protocol) + private def createLogManager(zkClient: ZkClient, brokerState: BrokerState): LogManager = { - val defaultLogConfig = LogConfig(segmentSize = config.logSegmentBytes, - segmentMs = config.logRollTimeMillis, - 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, - compact = config.logCleanupPolicy.trim.toLowerCase == "compact") - val defaultProps = defaultLogConfig.toProps + val defaultProps = KafkaServer.copyKafkaConfigToLog(config) + 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, @@ -346,17 +582,55 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg time = time) } - private def createOffsetManager(): OffsetManager = { - val offsetManagerConfig = OffsetManagerConfig( - maxMetadataSize = config.offsetMetadataMaxSize, - loadBufferSize = config.offsetsLoadBufferSize, - offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L, - offsetsTopicNumPartitions = config.offsetsTopicPartitions, - offsetsTopicReplicationFactor = config.offsetsTopicReplicationFactor, - offsetCommitTimeoutMs = config.offsetCommitTimeoutMs, - offsetCommitRequiredAcks = config.offsetCommitRequiredAcks) - new OffsetManager(offsetManagerConfig, replicaManager, zkClient, kafkaScheduler) + /** + * 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 cd64bbe56c99d..df521b3d93a1c 100644 --- a/core/src/main/scala/kafka/server/KafkaServerStartable.scala +++ b/core/src/main/scala/kafka/server/KafkaServerStartable.scala @@ -42,7 +42,8 @@ class KafkaServerStartable(val serverConfig: KafkaConfig) extends Logging { catch { case e: Throwable => fatal("Fatal error during KafkaServerStable shutdown. Prepare to halt", e) - System.exit(1) + // Calling exit() can lead to deadlock as exit() can be called multiple times. Force exit. + Runtime.getRuntime.halt(1) } } diff --git a/core/src/main/scala/kafka/server/LogOffsetMetadata.scala b/core/src/main/scala/kafka/server/LogOffsetMetadata.scala index a868334e0e53d..00b60fe152c68 100644 --- a/core/src/main/scala/kafka/server/LogOffsetMetadata.scala +++ b/core/src/main/scala/kafka/server/LogOffsetMetadata.scala @@ -26,7 +26,7 @@ object LogOffsetMetadata { class OffsetOrdering extends Ordering[LogOffsetMetadata] { override def compare(x: LogOffsetMetadata , y: LogOffsetMetadata ): Int = { - return x.offsetDiff(y).toInt + x.offsetDiff(y).toInt } } 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 old mode 100644 new mode 100755 index bf81a1ab88c14..9a9205f78ff20 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -17,26 +17,33 @@ package kafka.server -import scala.collection.{Seq, Set, mutable} -import kafka.api._ -import kafka.cluster.Broker -import java.util.concurrent.locks.ReentrantReadWriteLock -import kafka.utils.Utils._ +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 { +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() - def getTopicMetadata(topics: Set[String]) = { + 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] @@ -47,18 +54,21 @@ private[server] class MetadataCache { val partitionMetadata = partitionStateInfos.map { case (partitionId, partitionState) => val replicas = partitionState.allReplicas - val replicaInfo: Seq[Broker] = replicas.map(aliveBrokers.getOrElse(_, null)).filter(_ != null).toSeq - var leaderInfo: Option[Broker] = None - var isrInfo: Seq[Broker] = Nil + 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 { - leaderInfo = aliveBrokers.get(leader) - if (!leaderInfo.isDefined) + leaderBrokerInfo = aliveBrokers.get(leader) + if (!leaderBrokerInfo.isDefined) throw new LeaderNotAvailableException("Leader not available for %s".format(topicPartition)) - isrInfo = isr.map(aliveBrokers.getOrElse(_, null)).filter(_ != null) + 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(",")) @@ -68,7 +78,7 @@ private[server] class MetadataCache { new PartitionMetadata(partitionId, leaderInfo, replicaInfo, isrInfo, ErrorMapping.NoError) } catch { case e: Throwable => - debug("Error while fetching metadata for %s. Possible cause: %s".format(topicPartition, e.getMessage)) + 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]])) } @@ -133,6 +143,12 @@ private[server] class MetadataCache { } } + def contains(topic: String): Boolean = { + inReadLock(partitionMetadataLock) { + cache.contains(topic) + } + } + private def removePartitionInfo(topic: String, partitionId: Int) = { cache.get(topic) match { case Some(infos) => { @@ -146,4 +162,3 @@ private[server] class MetadataCache { } } } - diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala old mode 100644 new mode 100755 index 43eb2a35bb54d..0e613e782f71f --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -17,37 +17,39 @@ 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, LogConfig} +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.Properties 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 stale offsets. + * @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 @@ -61,7 +63,7 @@ import org.I0Itec.zkclient.ZkClient */ case class OffsetManagerConfig(maxMetadataSize: Int = OffsetManagerConfig.DefaultMaxMetadataSize, loadBufferSize: Int = OffsetManagerConfig.DefaultLoadBufferSize, - offsetsRetentionMs: Long = 24*60*60000L, + offsetsRetentionMs: Long = OffsetManagerConfig.DefaultOffsetRetentionMs, offsetsRetentionCheckIntervalMs: Long = OffsetManagerConfig.DefaultOffsetsRetentionCheckIntervalMs, offsetsTopicNumPartitions: Int = OffsetManagerConfig.DefaultOffsetsTopicNumPartitions, offsetsTopicSegmentBytes: Int = OffsetManagerConfig.DefaultOffsetsTopicSegmentBytes, @@ -73,10 +75,11 @@ case class OffsetManagerConfig(maxMetadataSize: Int = OffsetManagerConfig.Defaul object OffsetManagerConfig { val DefaultMaxMetadataSize = 4096 val DefaultLoadBufferSize = 5*1024*1024 + val DefaultOffsetRetentionMs = 24*60*60*1000L val DefaultOffsetsRetentionCheckIntervalMs = 600000L - val DefaultOffsetsTopicNumPartitions = 1 + val DefaultOffsetsTopicNumPartitions = 50 val DefaultOffsetsTopicSegmentBytes = 100*1024*1024 - val DefaultOffsetsTopicReplicationFactor = 1.toShort + val DefaultOffsetsTopicReplicationFactor = 3.toShort val DefaultOffsetsTopicCompressionCodec = NoCompressionCodec val DefaultOffsetCommitTimeoutMs = 5000 val DefaultOffsetCommitRequiredAcks = (-1).toShort @@ -90,13 +93,15 @@ class OffsetManager(val config: OffsetManagerConfig, /* 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) + private val offsetsTopicPartitionCount = getOffsetsTopicPartitionCount + + this.logIdent = "[Offset Manager on Broker " + replicaManager.config.brokerId + "]: " - scheduler.schedule(name = "offsets-cache-compactor", - fun = compact, + scheduler.schedule(name = "delete-expired-consumer-offsets", + fun = deleteExpiredOffsets, period = config.offsetsRetentionCheckIntervalMs, unit = TimeUnit.MILLISECONDS) @@ -112,61 +117,61 @@ class OffsetManager(val config: OffsetManagerConfig, } ) - private def compact() { - debug("Compacting offsets cache.") + private def deleteExpiredOffsets() { + debug("Collecting expired offsets.") val startMs = SystemTime.milliseconds - val staleOffsets = offsetsCache.filter(startMs - _._2.timestamp > config.offsetsRetentionMs) + val numExpiredOffsetsRemoved = cleanupOrLoadMutex synchronized { + val expiredOffsets = offsetsCache.filter { case (groupTopicPartition, offsetAndMetadata) => + offsetAndMetadata.expireTimestamp < startMs + } - debug("Found %d stale offsets (older than %d ms).".format(staleOffsets.size, config.offsetsRetentionMs)) + debug("Found %d expired offsets.".format(expiredOffsets.size)) - // delete the stale offsets from the table and generate tombstone messages to remove them from the log - val tombstonesForPartition = staleOffsets.map { case(groupTopicAndPartition, offsetAndMetadata) => - val offsetsPartition = partitionFor(groupTopicAndPartition.group) - trace("Removing stale offset and metadata for %s: %s".format(groupTopicAndPartition, offsetAndMetadata)) + // 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) + offsetsCache.remove(groupTopicAndPartition) - val commitKey = OffsetManager.offsetCommitKey(groupTopicAndPartition.group, - groupTopicAndPartition.topicPartition.topic, groupTopicAndPartition.topicPartition.partition) + val commitKey = OffsetManager.offsetCommitKey(groupTopicAndPartition.group, + groupTopicAndPartition.topicPartition.topic, groupTopicAndPartition.topicPartition.partition) - (offsetsPartition, new Message(bytes = null, key = commitKey)) - }.groupBy{ case (partition, tombstone) => 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 stale offsets during their own purge cycles. - val numRemoved = tombstonesForPartition.flatMap { case(offsetsPartition, tombstones) => - val partitionOpt = replicaManager.getPartition(OffsetManager.OffsetsTopicName, offsetsPartition) - partitionOpt.map { partition => - val appendPartition = TopicAndPartition(OffsetManager.OffsetsTopicName, offsetsPartition) - val messages = tombstones.map(_._2).toSeq + // 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)) + trace("Marked %d offsets in %s for deletion.".format(messages.size, appendPartition)) - try { - partition.appendMessagesToLeader(new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*)) - tombstones.size - } - catch { - case t: Throwable => - error("Failed to mark %d stale offsets for deletion in %s.".format(messages.size, appendPartition), t) - // ignore and continue - 0 + 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 + }.sum + } - debug("Removed %d stale offsets in %d milliseconds.".format(numRemoved, SystemTime.milliseconds - startMs)) + info("Removed %d expired offsets in %d milliseconds.".format(numExpiredOffsetsRemoved, SystemTime.milliseconds - startMs)) } - def offsetsTopicConfig: Properties = { - val props = new Properties - props.put(LogConfig.SegmentBytesProp, config.offsetsTopicSegmentBytes.toString) - props.put(LogConfig.CleanupPolicyProp, "compact") - props - } - def partitionFor(group: String): Int = Utils.abs(group.hashCode) % config.offsetsTopicNumPartitions + def partitionFor(group: String): Int = Utils.abs(group.hashCode) % offsetsTopicPartitionCount /** * Fetch the current offset for the given group/topic/partition from the underlying offsets storage. @@ -192,13 +197,93 @@ class OffsetManager(val config: OffsetManagerConfig, offsetsCache.put(key, offsetAndMetadata) } - def putOffsets(group: String, offsets: Map[TopicAndPartition, OffsetAndMetadata]) { - // this method is called _after_ the offsets have been durably appended to the commit log, so there is no need to - // check for current leadership as we do for the offset fetch - trace("Putting offsets %s for group %s in offsets partition %d.".format(offsets, group, partitionFor(group))) - offsets.foreach { case (topicAndPartition, offsetAndMetadata) => - putOffset(GroupTopicPartition(group, topicAndPartition), 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) } /** @@ -240,7 +325,7 @@ class OffsetManager(val config: OffsetManagerConfig, debug("Could not fetch offsets for group %s (not offset coordinator).".format(group)) topicPartitions.map { topicAndPartition => val groupTopicPartition = GroupTopicPartition(group, topicAndPartition) - (groupTopicPartition.topicPartition, OffsetMetadataAndError.NotOffsetManagerForGroup) + (groupTopicPartition.topicPartition, OffsetMetadataAndError.NotCoordinatorForGroup) }.toMap } } @@ -251,7 +336,7 @@ class OffsetManager(val config: OffsetManagerConfig, */ def loadOffsetsFromLog(offsetsPartition: Int) { - val topicPartition = TopicAndPartition(OffsetManager.OffsetsTopicName, offsetsPartition) + val topicPartition = TopicAndPartition(ConsumerCoordinator.OffsetsTopicName, offsetsPartition) loadingPartitions synchronized { if (loadingPartitions.contains(offsetsPartition)) { @@ -272,25 +357,36 @@ class OffsetManager(val config: OffsetManagerConfig, 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 - 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 { - val value = OffsetManager.readMessageValue(msgAndOffset.message.payload) - putOffset(key, value) - trace("Loaded offset %s for %s.".format(value, key)) + 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 } - currOffset = msgAndOffset.nextOffset } } @@ -312,7 +408,7 @@ class OffsetManager(val config: OffsetManagerConfig, } private def getHighWatermark(partitionId: Int): Long = { - val partitionOpt = replicaManager.getPartition(OffsetManager.OffsetsTopicName, partitionId) + val partitionOpt = replicaManager.getPartition(ConsumerCoordinator.OffsetsTopicName, partitionId) val hw = partitionOpt.map { partition => partition.leaderReplicaIfLocal().map(_.highWatermark.messageOffset).getOrElse(-1L) @@ -321,38 +417,51 @@ class OffsetManager(val config: OffsetManagerConfig, hw } - private def leaderIsLocal(partition: Int) = { getHighWatermark(partition) != -1L } + 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 clearOffsetsInPartition(offsetsPartition: Int) { - debug("Deleting offset entries belonging to [%s,%d].".format(OffsetManager.OffsetsTopicName, offsetsPartition)) - + 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) } + /** + * Gets the partition count of the offsets topic from ZooKeeper. + * If the topic does not exist, the configured partition count is returned. + */ + private def getOffsetsTopicPartitionCount = { + val topic = ConsumerCoordinator.OffsetsTopicName + val topicData = ZkUtils.getPartitionAssignmentForTopics(zkClient, Seq(topic)) + if (topicData(topic).nonEmpty) + topicData(topic).size + else + config.offsetsTopicNumPartitions + } } object OffsetManager { - val OffsetsTopicName = "__consumer_offsets" - private case class KeyAndValueSchemas(keySchema: Schema, valueSchema: Schema) - private val CURRENT_OFFSET_SCHEMA_VERSION = 0.toShort + 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), @@ -364,12 +473,24 @@ object OffsetManager { 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 VALUE_OFFSET_FIELD = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("offset") - private val VALUE_METADATA_FIELD = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("metadata") - private val VALUE_TIMESTAMP_FIELD = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("timestamp") + + 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)) + 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) @@ -386,7 +507,7 @@ object OffsetManager { * * @return key for offset commit message */ - def offsetCommitKey(group: String, topic: String, partition: Int, versionId: Short = 0): Array[Byte] = { + 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) @@ -404,12 +525,13 @@ object OffsetManager { * @param offsetAndMetadata consumer's current offset and metadata * @return payload for offset commit message */ - def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata): Array[Byte] = { + 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, offsetAndMetadata.offset) - value.set(VALUE_METADATA_FIELD, offsetAndMetadata.metadata) - value.set(VALUE_TIMESTAMP_FIELD, offsetAndMetadata.timestamp) - + 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) @@ -422,7 +544,7 @@ object OffsetManager { * @param buffer input byte-buffer * @return an GroupTopicPartition object */ - def readMessageKey(buffer: ByteBuffer): GroupTopicPartition = { + private def readMessageKey(buffer: ByteBuffer): GroupTopicPartition = { val version = buffer.getShort() val keySchema = schemaFor(version).keySchema val key = keySchema.read(buffer).asInstanceOf[Struct] @@ -440,19 +562,40 @@ object OffsetManager { * @param buffer input byte-buffer * @return an offset-metadata object from the message */ - def readMessageValue(buffer: ByteBuffer): OffsetAndMetadata = { - if(buffer == null) { // tombstone + 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] - val offset = value.get(VALUE_OFFSET_FIELD).asInstanceOf[Long] - val metadata = value.get(VALUE_METADATA_FIELD).asInstanceOf[String] - val timestamp = value.get(VALUE_TIMESTAMP_FIELD).asInstanceOf[Long] - - OffsetAndMetadata(offset, metadata, timestamp) + MessageValueStructAndVersion(value, version) } } @@ -461,7 +604,7 @@ object OffsetManager { 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.readMessageValue(ByteBuffer.wrap(value)).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) @@ -471,6 +614,8 @@ object OffsetManager { } +case class MessageValueStructAndVersion(value: Struct, version: Short) + case class GroupTopicPartition(group: String, topicPartition: TopicAndPartition) { def this(group: String, topic: String, partition: Int) = diff --git a/core/src/main/scala/kafka/server/ProducerRequestPurgatory.scala b/core/src/main/scala/kafka/server/ProducerRequestPurgatory.scala deleted file mode 100644 index d4a7d4a79b442..0000000000000 --- a/core/src/main/scala/kafka/server/ProducerRequestPurgatory.scala +++ /dev/null @@ -1,69 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.server - -import kafka.metrics.KafkaMetricsGroup -import kafka.utils.Pool -import kafka.network.{BoundedByteBufferSend, RequestChannel} - -import java.util.concurrent.TimeUnit - -/** - * The purgatory holding delayed producer requests - */ -class ProducerRequestPurgatory(replicaManager: ReplicaManager, offsetManager: OffsetManager, requestChannel: RequestChannel) - extends RequestPurgatory[DelayedProduce](replicaManager.config.brokerId, replicaManager.config.producerPurgatoryPurgeIntervalRequests) { - this.logIdent = "[ProducerRequestPurgatory-%d] ".format(replicaManager.config.brokerId) - - private class DelayedProducerRequestMetrics(keyLabel: String = DelayedRequestKey.globalLabel) extends KafkaMetricsGroup { - val expiredRequestMeter = newMeter(keyLabel + "ExpiresPerSecond", "requests", TimeUnit.SECONDS) - } - - private val producerRequestMetricsForKey = { - val valueFactory = (k: DelayedRequestKey) => new DelayedProducerRequestMetrics(k.keyLabel + "-") - new Pool[DelayedRequestKey, DelayedProducerRequestMetrics](Some(valueFactory)) - } - - private val aggregateProduceRequestMetrics = new DelayedProducerRequestMetrics - - private def recordDelayedProducerKeyExpired(key: DelayedRequestKey) { - val keyMetrics = producerRequestMetricsForKey.getAndMaybePut(key) - List(keyMetrics, aggregateProduceRequestMetrics).foreach(_.expiredRequestMeter.mark()) - } - - /** - * Check if a specified delayed fetch request is satisfied - */ - def checkSatisfied(delayedProduce: DelayedProduce) = delayedProduce.isSatisfied(replicaManager) - - /** - * When a delayed produce request expires answer it with possible time out error codes - */ - def expire(delayedProduce: DelayedProduce) { - debug("Expiring produce request %s.".format(delayedProduce.produce)) - for ((topicPartition, responseStatus) <- delayedProduce.partitionStatus if responseStatus.acksPending) - recordDelayedProducerKeyExpired(new TopicPartitionRequestKey(topicPartition)) - respond(delayedProduce) - } - - // TODO: purgatory should not be responsible for sending back the responses - def respond(delayedProduce: DelayedProduce) { - val response = delayedProduce.respond(offsetManager) - requestChannel.sendResponse(new RequestChannel.Response(delayedProduce.request, new BoundedByteBufferSend(response))) - } -} diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala b/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala index 351dbbad3bdb7..779876b5572a3 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala @@ -17,14 +17,23 @@ package kafka.server -import kafka.cluster.Broker +import kafka.cluster.BrokerEndPoint +import org.apache.kafka.common.metrics.Metrics +import org.apache.kafka.common.utils.Time -class ReplicaFetcherManager(private val brokerConfig: KafkaConfig, private val replicaMgr: ReplicaManager) +class ReplicaFetcherManager(brokerConfig: KafkaConfig, replicaMgr: ReplicaManager, metrics: Metrics, time: Time, threadNamePrefix: Option[String] = None) extends AbstractFetcherManager("ReplicaFetcherManager on broker " + brokerConfig.brokerId, "Replica", brokerConfig.numReplicaFetchers) { - override def createFetcherThread(fetcherId: Int, sourceBroker: Broker): AbstractFetcherThread = { - new ReplicaFetcherThread("ReplicaFetcherThread-%d-%d".format(fetcherId, sourceBroker.id), sourceBroker, brokerConfig, replicaMgr) + override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): AbstractFetcherThread = { + val threadName = threadNamePrefix match { + case None => + "ReplicaFetcherThread-%d-%d".format(fetcherId, sourceBroker.id) + case Some(p) => + "%s:ReplicaFetcherThread-%d-%d".format(p, fetcherId, sourceBroker.id) + } + new ReplicaFetcherThread(threadName, sourceBroker, brokerConfig, + replicaMgr, metrics, time) } def shutdown() { @@ -32,4 +41,4 @@ class ReplicaFetcherManager(private val brokerConfig: KafkaConfig, private val r closeAllFetchers() info("shutdown completed") } -} \ No newline at end of file +} diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 6879e73028218..82a60018acdd3 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -17,44 +17,97 @@ package kafka.server +import java.net.SocketTimeoutException + import kafka.admin.AdminUtils -import kafka.cluster.Broker +import kafka.cluster.BrokerEndPoint import kafka.log.LogConfig import kafka.message.ByteBufferMessageSet -import kafka.api.{OffsetRequest, FetchResponsePartitionData} +import kafka.api.KAFKA_090 import kafka.common.{KafkaStorageException, TopicAndPartition} +import ReplicaFetcherThread._ +import org.apache.kafka.clients.{ManualMetadataUpdater, NetworkClient, ClientRequest, ClientResponse} +import org.apache.kafka.common.network.{Selectable, ChannelBuilders, NetworkReceive, Selector} +import org.apache.kafka.common.requests.{ListOffsetResponse, FetchResponse, RequestSend, AbstractRequest, ListOffsetRequest} +import org.apache.kafka.common.requests.{FetchRequest => JFetchRequest} +import org.apache.kafka.common.{Node, TopicPartition} +import org.apache.kafka.common.metrics.Metrics +import org.apache.kafka.common.protocol.{Errors, ApiKeys} +import org.apache.kafka.common.security.ssl.SSLFactory +import org.apache.kafka.common.utils.Time + +import scala.collection.{JavaConverters, Map, mutable} +import JavaConverters._ -class ReplicaFetcherThread(name:String, - sourceBroker: Broker, +class ReplicaFetcherThread(name: String, + sourceBroker: BrokerEndPoint, brokerConfig: KafkaConfig, - replicaMgr: ReplicaManager) + replicaMgr: ReplicaManager, + metrics: Metrics, + time: Time) extends AbstractFetcherThread(name = name, clientId = name, sourceBroker = sourceBroker, - socketTimeout = brokerConfig.replicaSocketTimeoutMs, - socketBufferSize = brokerConfig.replicaSocketReceiveBufferBytes, - fetchSize = brokerConfig.replicaFetchMaxBytes, - fetcherBrokerId = brokerConfig.brokerId, - maxWait = brokerConfig.replicaFetchWaitMaxMs, - minBytes = brokerConfig.replicaFetchMinBytes, + fetchBackOffMs = brokerConfig.replicaFetchBackoffMs, isInterruptible = false) { + type REQ = FetchRequest + type PD = PartitionData + + private val fetchRequestVersion: Short = if (brokerConfig.interBrokerProtocolVersion.onOrAfter(KAFKA_090)) 1 else 0 + private val socketTimeout: Int = brokerConfig.replicaSocketTimeoutMs + private val replicaId = brokerConfig.brokerId + private val maxWait = brokerConfig.replicaFetchWaitMaxMs + private val minBytes = brokerConfig.replicaFetchMinBytes + private val fetchSize = brokerConfig.replicaFetchMaxBytes + + private def clientId = name + + private val sourceNode = new Node(sourceBroker.id, sourceBroker.host, sourceBroker.port) + + private val networkClient = { + val selector = new Selector( + NetworkReceive.UNLIMITED, + brokerConfig.connectionsMaxIdleMs, + metrics, + time, + "replica-fetcher", + Map("broker-id" -> sourceBroker.id.toString).asJava, + false, + ChannelBuilders.create(brokerConfig.interBrokerSecurityProtocol, SSLFactory.Mode.CLIENT, brokerConfig.channelConfigs) + ) + new NetworkClient( + selector, + new ManualMetadataUpdater(), + clientId, + 1, + 0, + Selectable.USE_DEFAULT_BUFFER_SIZE, + brokerConfig.replicaSocketReceiveBufferBytes, + brokerConfig.requestTimeoutMs + ) + } + + override def shutdown(): Unit = { + super.shutdown() + networkClient.close() + } + // process fetched data - def processPartitionData(topicAndPartition: TopicAndPartition, fetchOffset: Long, partitionData: FetchResponsePartitionData) { + def processPartitionData(topicAndPartition: TopicAndPartition, fetchOffset: Long, partitionData: PartitionData) { try { - val topic = topicAndPartition.topic - val partitionId = topicAndPartition.partition + val TopicAndPartition(topic, partitionId) = topicAndPartition val replica = replicaMgr.getReplica(topic, partitionId).get - val messageSet = partitionData.messages.asInstanceOf[ByteBufferMessageSet] + val messageSet = partitionData.toByteBufferMessageSet 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.messageOffset, topicAndPartition, messageSet.sizeInBytes, partitionData.hw)) + .format(replica.brokerId, replica.logEndOffset.messageOffset, topicAndPartition, messageSet.sizeInBytes, partitionData.highWatermark)) 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.messageOffset, messageSet.sizeInBytes, topicAndPartition)) - val followerHighWatermark = replica.logEndOffset.messageOffset.min(partitionData.hw) + val followerHighWatermark = replica.logEndOffset.messageOffset.min(partitionData.highWatermark) // 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 @@ -84,13 +137,15 @@ 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) + val leaderEndOffset: Long = earliestOrLatestOffset(topicAndPartition, ListOffsetRequest.LATEST_TIMESTAMP, + brokerConfig.brokerId) + 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.props.props, AdminUtils.fetchTopicConfig(replicaMgr.zkClient, - topicAndPartition.topic)).uncleanLeaderElectionEnable) { + if (!LogConfig.fromProps(brokerConfig.originals, AdminUtils.fetchEntityConfig(replicaMgr.zkClient, + ConfigType.Topic, 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" @@ -109,16 +164,98 @@ 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) - replicaMgr.logManager.truncateFullyAndStartAt(topicAndPartition, leaderStartOffset) + val leaderStartOffset: Long = earliestOrLatestOffset(topicAndPartition, ListOffsetRequest.EARLIEST_TIMESTAMP, + 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) 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) + } + + protected def fetch(fetchRequest: FetchRequest): Map[TopicAndPartition, PartitionData] = { + val clientResponse = sendRequest(ApiKeys.FETCH, Some(fetchRequestVersion), fetchRequest.underlying) + new FetchResponse(clientResponse.responseBody).responseData.asScala.map { case (key, value) => + TopicAndPartition(key.topic, key.partition) -> new PartitionData(value) + } + } + + private def sendRequest(apiKey: ApiKeys, apiVersion: Option[Short], request: AbstractRequest): ClientResponse = { + import kafka.utils.NetworkClientBlockingOps._ + val header = apiVersion.fold(networkClient.nextRequestHeader(apiKey))(networkClient.nextRequestHeader(apiKey, _)) + try { + if (!networkClient.blockingReady(sourceNode, socketTimeout)(time)) + throw new SocketTimeoutException(s"Failed to connect within $socketTimeout ms") + else { + val send = new RequestSend(sourceBroker.id.toString, header, request.toStruct) + val clientRequest = new ClientRequest(time.milliseconds(), true, send, null) + networkClient.blockingSendAndReceive(clientRequest, socketTimeout)(time).getOrElse { + throw new SocketTimeoutException(s"No response received within $socketTimeout ms") + } + } + } + catch { + case e: Throwable => + networkClient.close(sourceBroker.id.toString) + throw e + } + } + + private def earliestOrLatestOffset(topicAndPartition: TopicAndPartition, earliestOrLatest: Long, consumerId: Int): Long = { + val topicPartition = new TopicPartition(topicAndPartition.topic, topicAndPartition.partition) + val partitions = Map( + topicPartition -> new ListOffsetRequest.PartitionData(earliestOrLatest, 1) + ) + val request = new ListOffsetRequest(consumerId, partitions.asJava) + val clientResponse = sendRequest(ApiKeys.LIST_OFFSETS, None, request) + val response = new ListOffsetResponse(clientResponse.responseBody) + val partitionData = response.responseData.get(topicPartition) + Errors.forCode(partitionData.errorCode) match { + case Errors.NONE => partitionData.offsets.asScala.head + case errorCode => throw errorCode.exception + } + } + + protected def buildFetchRequest(partitionMap: Map[TopicAndPartition, PartitionFetchState]): FetchRequest = { + val requestMap = mutable.Map.empty[TopicPartition, JFetchRequest.PartitionData] + + partitionMap.foreach { case ((TopicAndPartition(topic, partition), partitionFetchState)) => + if (partitionFetchState.isActive) + requestMap(new TopicPartition(topic, partition)) = new JFetchRequest.PartitionData(partitionFetchState.offset, fetchSize) + } + + new FetchRequest(new JFetchRequest(replicaId, maxWait, minBytes, requestMap.asJava)) + } + +} + +object ReplicaFetcherThread { + + private[server] class FetchRequest(val underlying: JFetchRequest) extends AbstractFetcherThread.FetchRequest { + def isEmpty: Boolean = underlying.fetchData.isEmpty + def offset(topicAndPartition: TopicAndPartition): Long = + underlying.fetchData.asScala(new TopicPartition(topicAndPartition.topic, topicAndPartition.partition)).offset + } + + private[server] class PartitionData(val underlying: FetchResponse.PartitionData) extends AbstractFetcherThread.PartitionData { + + def errorCode: Short = underlying.errorCode + + def toByteBufferMessageSet: ByteBufferMessageSet = new ByteBufferMessageSet(underlying.recordSet) + + def highWatermark: Long = underlying.highWatermark + + def exception: Option[Throwable] = Errors.forCode(errorCode) match { + case Errors.NONE => None + case e => Some(e.exception) + } + + } + } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 78b7514cc1095..c0fec67477868 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -16,56 +16,111 @@ */ package kafka.server +import java.io.{File, IOException} +import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic.AtomicBoolean + +import com.yammer.metrics.core.Gauge import kafka.api._ +import kafka.cluster.{BrokerEndPoint, Partition, Replica} import kafka.common._ -import kafka.utils._ -import kafka.cluster.{Broker, Partition, Replica} -import kafka.log.LogManager -import kafka.metrics.KafkaMetricsGroup import kafka.controller.KafkaController -import kafka.common.TopicAndPartition -import kafka.message.MessageSet +import kafka.log.{LogAppendInfo, LogManager} +import kafka.message.{ByteBufferMessageSet, MessageSet} +import kafka.metrics.KafkaMetricsGroup +import kafka.utils._ +import org.I0Itec.zkclient.ZkClient +import org.apache.kafka.common.metrics.Metrics +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.utils.{Time => JTime} -import java.util.concurrent.atomic.AtomicBoolean -import java.io.{IOException, File} -import java.util.concurrent.TimeUnit -import scala.Predef._ import scala.collection._ -import scala.collection.mutable.HashMap -import scala.collection.Map -import scala.collection.Set -import scala.Some -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]]) + } +} -object ReplicaManager { - val HighWatermarkFilename = "replication-offset-checkpoint" +/* + * 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) + } } -case class PartitionDataAndOffset(data: FetchResponsePartitionData, offset: LogOffsetMetadata) +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) { -class ReplicaManager(val config: KafkaConfig, - time: Time, - val zkClient: ZkClient, + override def toString = { + "updated leaders: [%s], updated followers: [%s], update results: [%s], global error: [%d]" + .format(updatedLeaders, updatedFollowers, responseMap, errorCode) + } +} + +object ReplicaManager { + val HighWatermarkFilename = "replication-offset-checkpoint" +} + +class ReplicaManager(val config: KafkaConfig, + metrics: Metrics, + time: Time, + jTime: JTime, + val zkClient: ZkClient, scheduler: Scheduler, val logManager: LogManager, - val isShuttingDown: AtomicBoolean ) extends Logging with KafkaMetricsGroup { + val isShuttingDown: AtomicBoolean, + threadNamePrefix: Option[String] = None) extends Logging with KafkaMetricsGroup { /* epoch of the controller that last changed the leader */ @volatile var controllerEpoch: Int = KafkaController.InitialControllerEpoch - 1 private val localBrokerId = config.brokerId private val allPartitions = new Pool[(String, Int), Partition] private val replicaStateChangeLock = new Object - val replicaFetcherManager = new ReplicaFetcherManager(config, this) + val replicaFetcherManager = new ReplicaFetcherManager(config, this, metrics, jTime, threadNamePrefix) private val highWatermarkCheckPointThreadStarted = new AtomicBoolean(false) 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 = KafkaController.stateChangeLogger + private val isrChangeSet: mutable.Set[TopicAndPartition] = new mutable.HashSet[TopicAndPartition]() - var producerRequestPurgatory: ProducerRequestPurgatory = null - var fetchRequestPurgatory: FetchRequestPurgatory = null + val delayedProducePurgatory = new DelayedOperationPurgatory[DelayedProduce]( + purgatoryName = "Produce", config.brokerId, config.producerPurgatoryPurgeIntervalRequests) + val delayedFetchPurgatory = new DelayedOperationPurgatory[DelayedFetch]( + purgatoryName = "Fetch", config.brokerId, config.fetchPurgatoryPurgeIntervalRequests) newGauge( "LeaderCount", @@ -99,43 +154,49 @@ class ReplicaManager(val config: KafkaConfig, scheduler.schedule("highwatermark-checkpoint", checkpointHighWatermarks, period = config.replicaHighWatermarkCheckpointIntervalMs, unit = TimeUnit.MILLISECONDS) } - /** - * Initialize the replica manager with the request purgatory - * - * TODO: will be removed in 0.9 where we refactor server structure - */ + def recordIsrChange(topicAndPartition: TopicAndPartition) { + isrChangeSet synchronized { + isrChangeSet += topicAndPartition + } + } - def initWithRequestPurgatory(producerRequestPurgatory: ProducerRequestPurgatory, fetchRequestPurgatory: FetchRequestPurgatory) { - this.producerRequestPurgatory = producerRequestPurgatory - this.fetchRequestPurgatory = fetchRequestPurgatory + def maybePropagateIsrChanges() { + isrChangeSet synchronized { + if (isrChangeSet.nonEmpty) { + ReplicationUtils.propagateIsrChanges(zkClient, isrChangeSet) + isrChangeSet.clear() + } + } } /** - * Unblock some delayed produce requests with the request key + * 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 unblockDelayedProduceRequests(key: DelayedRequestKey) { - val satisfied = producerRequestPurgatory.update(key) - debug("Request key %s unblocked %d producer requests." - .format(key.keyLabel, satisfied.size)) - - // send any newly unblocked responses - satisfied.foreach(producerRequestPurgatory.respond(_)) + def tryCompleteDelayedProduce(key: DelayedOperationKey) { + val completed = delayedProducePurgatory.checkAndComplete(key) + debug("Request key %s unblocked %d producer requests.".format(key.keyLabel, completed)) } /** - * Unblock some delayed fetch requests with the request key + * 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 unblockDelayedFetchRequests(key: DelayedRequestKey) { - val satisfied = fetchRequestPurgatory.update(key) - debug("Request key %s unblocked %d fetch requests.".format(key.keyLabel, satisfied.size)) - - // send any newly unblocked responses - satisfied.foreach(fetchRequestPurgatory.respond(_)) + def tryCompleteDelayedFetch(key: DelayedOperationKey) { + val completed = delayedFetchPurgatory.checkAndComplete(key) + debug("Request key %s unblocked %d fetch requests.".format(key.keyLabel, completed)) } def startup() { // start ISR expiration thread scheduler.schedule("isr-expiration", maybeShrinkIsr, period = config.replicaLagTimeMaxMs, unit = TimeUnit.MILLISECONDS) + scheduler.schedule("isr-change-propagation", maybePropagateIsrChanges, period = 5000, unit = TimeUnit.MILLISECONDS) } def stopReplica(topic: String, partitionId: Int, deletePartition: Boolean): Short = { @@ -208,7 +269,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)) } @@ -237,74 +298,259 @@ class ReplicaManager(val config: KafkaConfig, } /** - * Read from all the offset details given and return a map of - * (topic, partition) -> PartitionData + * 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 readMessageSets(fetchRequest: FetchRequest) = { - val isFetchFromFollower = fetchRequest.isFromFollower - fetchRequest.requestInfo.map - { - case (TopicAndPartition(topic, partition), PartitionFetchInfo(offset, fetchSize)) => - val partitionDataAndOffsetInfo = - try { - val (fetchInfo, highWatermark) = readMessageSet(topic, partition, offset, fetchSize, fetchRequest.replicaId) - BrokerTopicStats.getBrokerTopicStats(topic).bytesOutRate.mark(fetchInfo.messageSet.sizeInBytes) - BrokerTopicStats.getBrokerAllTopicsStats.bytesOutRate.mark(fetchInfo.messageSet.sizeInBytes) - if (isFetchFromFollower) { - debug("Partition [%s,%d] received fetch request from follower %d" - .format(topic, partition, fetchRequest.replicaId)) - } - new PartitionDataAndOffset(new FetchResponsePartitionData(ErrorMapping.NoError, highWatermark, fetchInfo.messageSet), fetchInfo.fetchOffset) - } 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 PartitionDataAndOffset(new FetchResponsePartitionData(ErrorMapping.codeFor(utpe.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty), LogOffsetMetadata.UnknownOffsetMetadata) - 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 PartitionDataAndOffset(new FetchResponsePartitionData(ErrorMapping.codeFor(nle.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty), LogOffsetMetadata.UnknownOffsetMetadata) - 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. Possible cause: %s" - .format(topic, partition, offset, if (isFetchFromFollower) "follower" else "consumer", fetchRequest.correlationId, t.getMessage)) - new PartitionDataAndOffset(new FetchResponsePartitionData(ErrorMapping.codeFor(t.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty), LogOffsetMetadata.UnknownOffsetMetadata) + 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)) } - (TopicAndPartition(topic, partition), partitionDataAndOffsetInfo) + + 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 */ - private def readMessageSet(topic: String, - partition: Int, - offset: Long, - maxSize: Int, - fromReplicaId: Int): (FetchDataInfo, Long) = { - // check if the current broker is the leader for the partitions - val localReplica = if(fromReplicaId == Request.DebuggingConsumerId) - getReplicaOrException(topic, partition) - else - getLeaderReplicaIfLocal(topic, partition) - trace("Fetching log segment for topic, partition, offset, size = " + (topic, partition, offset, maxSize)) - val maxOffsetOpt = - if (Request.isValidBrokerId(fromReplicaId)) - None - else - Some(localReplica.highWatermark.messageOffset) - val fetchInfo = localReplica.log match { - case Some(log) => - log.read(offset, maxSize, maxOffsetOpt) - case None => - error("Leader for partition [%s,%d] does not have a local log".format(topic, partition)) - FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty) + 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) } - (fetchInfo, localReplica.highWatermark.messageOffset) } def maybeUpdateMetadataCache(updateMetadataRequest: UpdateMetadataRequest, metadataCache: MetadataCache) { @@ -323,30 +569,29 @@ class ReplicaManager(val config: KafkaConfig, } } - def becomeLeaderOrFollower(leaderAndISRRequest: LeaderAndIsrRequest, - offsetManager: OffsetManager): (collection.Map[(String, Int), Short], Short) = { + 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 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 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. @@ -370,14 +615,19 @@ class ReplicaManager(val config: KafkaConfig, } } - 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, offsetManager) - if (!partitionsToBeFollower.isEmpty) - makeFollowers(controllerId, controllerEpoch, partitionsToBeFollower, leaderAndISRRequest.leaders, leaderAndISRRequest.correlationId, responseMap, offsetManager) + 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 @@ -386,7 +636,7 @@ class ReplicaManager(val config: KafkaConfig, hwThreadInitialized = true } replicaFetcherManager.shutdownIdleFetcherThreads() - (responseMap, ErrorMapping.NoError) + BecomeLeaderOrFollowerResult(responseMap, partitionsBecomeLeader, partitionsBecomeFollower, ErrorMapping.NoError) } } } @@ -402,10 +652,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], - offsetManager: OffsetManager) = { + 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") @@ -424,7 +675,7 @@ class ReplicaManager(val config: KafkaConfig, } // Update the partition information to be the leader partitionState.foreach{ case (partition, partitionStateInfo) => - partition.makeLeader(controllerId, partitionStateInfo, correlationId, offsetManager)} + partition.makeLeader(controllerId, partitionStateInfo, correlationId)} } catch { case e: Throwable => @@ -443,6 +694,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 } /* @@ -461,9 +714,12 @@ 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], - offsetManager: OffsetManager) { + 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") @@ -473,17 +729,18 @@ class ReplicaManager(val config: KafkaConfig, for (partition <- partitionState.keys) responseMap.put((partition.topic, partition.partitionId), ErrorMapping.NoError) - try { + val partitionsToMakeFollower: mutable.Set[Partition] = mutable.Set() - var partitionsToMakeFollower: Set[Partition] = Set() + try { - // TODO: Delete leaders from LeaderAndIsrRequest in 0.8.1 + // 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, offsetManager)) + 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 " + @@ -493,10 +750,13 @@ class ReplicaManager(val config: KafkaConfig, 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 aborted the become-follower state change with correlation id %d from " + - "controller %d epoch %d for partition [%s,%d] since new leader %d is not currently available") + 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() } } @@ -550,39 +810,36 @@ 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") - allPartitions.values.foreach(partition => partition.maybeShrinkIsr(config.replicaLagTimeMaxMs, config.replicaLagMaxMessages)) + allPartitions.values.foreach(partition => partition.maybeShrinkIsr(config.replicaLagTimeMaxMs)) } - def updateReplicaLEOAndPartitionHW(topic: String, partitionId: Int, replicaId: Int, offset: LogOffsetMetadata) = { - getPartition(topic, partitionId) match { - case Some(partition) => - partition.getReplica(replicaId) match { - case Some(replica) => - replica.logEndOffset = offset - // check if we need to update HW and expand Isr - partition.updateLeaderHWAndMaybeExpandIsr(replicaId) - debug("Recorded follower %d position %d for partition [%s,%d].".format(replicaId, offset.messageOffset, 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,%d]").format(localBrokerId, replicaId, - offset.messageOffset, partition.assignedReplicas().map(_.brokerId).mkString(","), topic, partitionId)) - - } - case None => - 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)) + } } } private def getLeaderPartitions() : List[Partition] = { allPartitions.values.filter(_.leaderReplicaIfLocal().isDefined).toList } - /** - * Flushes the highwatermark value for all partitions to the highwatermark file - */ + + // 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.getParentFile.getAbsolutePath) @@ -598,10 +855,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() + 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 cf3ed4c8f197d..0000000000000 --- a/core/src/main/scala/kafka/server/RequestPurgatory.scala +++ /dev/null @@ -1,325 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.server - -import kafka.network._ -import kafka.utils._ -import kafka.metrics.KafkaMetricsGroup - -import java.util -import java.util.concurrent._ -import java.util.concurrent.atomic._ -import scala.collection._ - -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 - * val isSatisfiedByMe = checkAndMaybeWatch(delayedRequest) - * we will check if a request is satisfied already, and if not add the request for watch on all its 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](brokerId: Int = 0, purgeInterval: Int = 1000) - extends Logging with KafkaMetricsGroup { - - /* a list of requests watching each key */ - private val watchersForKey = new Pool[Any, Watchers](Some((key: Any) => new Watchers)) - - /* 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) - - newGauge( - "PurgatorySize", - new Gauge[Int] { - def value = watched() - } - ) - - newGauge( - "NumDelayedRequests", - new Gauge[Int] { - def value = delayed() - } - ) - - expirationThread.start() - - /** - * Try to add the request for watch on all keys. Return true iff the request is - * satisfied and the satisfaction is done by the caller. - * - * Requests can be watched on only a few of the keys if it is found satisfied when - * trying to add it to each one of the keys. In this case the request is still treated as satisfied - * and hence no longer watched. Those already added elements will be later purged by the expire reaper. - */ - def checkAndMaybeWatch(delayedRequest: T): Boolean = { - for(key <- delayedRequest.keys) { - val lst = watchersFor(key) - if(!lst.checkAndMaybeAdd(delayedRequest)) { - if(delayedRequest.satisfied.compareAndSet(false, true)) - return true - else - return false - } - } - - // if it is indeed watched, add to the expire queue also - expiredRequestReaper.enqueue(delayedRequest) - - false - } - - /** - * Update any watchers and return a list of newly satisfied requests. - */ - def update(key: Any): Seq[T] = { - val w = watchersForKey.get(key) - if(w == null) - Seq.empty - else - w.collectSatisfiedRequests() - } - - /* - * Return the size of the watched lists in the purgatory, which is the size of watch lists. - * Since an operation 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() = watchersForKey.values.map(_.watched).sum - - /* - * Return the number of requests in the expiry reaper's queue - */ - def delayed() = expiredRequestReaper.delayed() - - /* - * Return the watch list for the given watch key - */ - private def watchersFor(key: Any) = watchersForKey.getAndMaybePut(key) - - /** - * Check if this delayed request is already satisfied - */ - protected def checkSatisfied(request: T): Boolean - - /** - * Handle an expired delayed request - */ - protected def expire(delayed: T) - - /** - * Shutdown the expire reaper 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] - - // return the size of the watch list - def watched() = requests.size() - - // potentially add the element to watch if it is not satisfied yet - def checkAndMaybeAdd(t: T): Boolean = { - synchronized { - // if it is already satisfied, do not add to the watch list - if (t.satisfied.get) - return false - // synchronize on the delayed request to avoid any race condition - // with expire and update threads on client-side. - if(t synchronized checkSatisfied(t)) { - return false - } - requests.add(t) - return true - } - } - - // traverse the list and purge satisfied elements - 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 - } - } - - // traverse the list and try to satisfy watched elements - def collectSatisfiedRequests(): 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(curr) - if(satisfied) { - iter.remove() - val updated = curr.satisfied.compareAndSet(false, true) - if(updated == true) { - response += curr - } - } - } - } - } - 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 running = new AtomicBoolean(true) - private val shutdownLatch = new CountDownLatch(1) - - private val delayedQueue = new DelayQueue[T] - - def delayed() = delayedQueue.size() - - /** Main loop for the expiry thread */ - def run() { - while(running.get) { - try { - val curr = pollExpired() - if (curr != null) { - curr synchronized { - expire(curr) - } - } - // see if we need to purge the watch lists - if (RequestPurgatory.this.watched() >= purgeInterval) { - debug("Begin purging watch lists") - val numPurgedFromWatchers = watchersForKey.values.map(_.purgeSatisfied()).sum - debug("Purged %d elements from watch lists.".format(numPurgedFromWatchers)) - } - // see if we need to purge the delayed request queue - if (delayed() >= purgeInterval) { - debug("Begin purging delayed queue") - val purged = purgeSatisfied() - debug("Purged %d requests from delayed queue.".format(purged)) - } - } catch { - case e: Exception => - error("Error in long poll expiry thread: ", e) - } - } - shutdownLatch.countDown() - } - - /** Add a request to be expired */ - def enqueue(t: T) { - delayedQueue.add(t) - } - - /** Shutdown the expiry thread*/ - def shutdown() { - debug("Shutting down.") - running.set(false) - shutdownLatch.await() - debug("Shut down complete.") - } - - /** - * Get the next expired event - */ - private def pollExpired(): T = { - while(true) { - val curr = delayedQueue.poll(200L, TimeUnit.MILLISECONDS) - if (curr == null) - return null.asInstanceOf[T] - val updated = curr.satisfied.compareAndSet(false, true) - if(updated) { - return curr - } - } - throw new RuntimeException("This should not happen") - } - - /** - * Delete all satisfied events from the delay queue and the watcher lists - */ - private def purgeSatisfied(): Int = { - var purged = 0 - - // purge the delayed queue - val iter = delayedQueue.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/ThrottledResponse.scala b/core/src/main/scala/kafka/server/ThrottledResponse.scala new file mode 100644 index 0000000000000..214fa1f5c837a --- /dev/null +++ b/core/src/main/scala/kafka/server/ThrottledResponse.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.server + +import java.util.concurrent.{TimeUnit, Delayed} + +import org.apache.kafka.common.utils.Time + + +/** + * Represents a request whose response has been delayed. + * @param time @Time instance to use + * @param throttleTimeMs delay associated with this request + * @param callback Callback to trigger after delayTimeMs milliseconds + */ +private[server] class ThrottledResponse(val time: Time, val throttleTimeMs: Int, callback: Int => Unit) extends Delayed { + val endTime = time.milliseconds + throttleTimeMs + + def execute() = callback(throttleTimeMs) + + override def getDelay(unit: TimeUnit): Long = { + unit.convert(endTime - time.milliseconds, TimeUnit.MILLISECONDS) + } + + override def compareTo(d: Delayed): Int = { + val other = d.asInstanceOf[ThrottledResponse] + if (this.endTime < other.endTime) -1 + else if (this.endTime > other.endTime) 1 + else 0 + } +} 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 a75818a724ff3..b283e0a37b410 --- a/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala +++ b/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala @@ -17,8 +17,8 @@ package kafka.server import kafka.utils.ZkUtils._ -import kafka.utils.Utils._ -import kafka.utils.{Json, SystemTime, Logging} +import kafka.utils.CoreUtils._ +import kafka.utils.{Json, SystemTime, Logging, ZKCheckedEphemeral} import org.I0Itec.zkclient.exception.ZkNodeExistsException import org.I0Itec.zkclient.IZkDataListener import kafka.controller.ControllerContext @@ -56,7 +56,7 @@ class ZookeeperLeaderElector(controllerContext: ControllerContext, case None => -1 } } - + def elect: Boolean = { val timestamp = SystemTime.milliseconds.toString val electString = Json.encode(Map("version" -> 1, "brokerid" -> brokerId, "timestamp" -> timestamp)) @@ -73,9 +73,10 @@ class ZookeeperLeaderElector(controllerContext: ControllerContext, } try { - createEphemeralPathExpectConflictHandleZKBug(controllerContext.zkClient, electionPath, electString, brokerId, - (controllerString : String, leaderId : Any) => KafkaController.parseControllerId(controllerString) == leaderId.asInstanceOf[Int], - controllerContext.zkSessionTimeout) + val zkCheckedEphemeral = new ZKCheckedEphemeral(electionPath, + electString, + controllerContext.zkConnection.getZookeeper) + zkCheckedEphemeral.create() info(brokerId + " successfully elected as leader") leaderId = brokerId onBecomingLeader() @@ -119,8 +120,12 @@ class ZookeeperLeaderElector(controllerContext: ControllerContext, @throws(classOf[Exception]) def handleDataChange(dataPath: String, data: Object) { inLock(controllerContext.controllerLock) { + val amILeaderBeforeDataChange = amILeader leaderId = KafkaController.parseControllerId(data.toString) info("New leader is %d".format(leaderId)) + // The old leader needs to resign leadership if it is no longer the leader + if (amILeaderBeforeDataChange && !amILeader) + onResigningAsLeader() } } diff --git a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala old mode 100644 new mode 100755 index 323fc8566d974..de4900dd3c8f2 --- a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleConsumer.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, @@ -17,90 +17,247 @@ package kafka.tools -import scala.collection.JavaConversions._ -import org.I0Itec.zkclient._ -import joptsimple._ -import java.util.Properties -import java.util.Random import java.io.PrintStream +import java.util.{Properties, Random} +import joptsimple._ +import kafka.consumer._ import kafka.message._ -import kafka.serializer._ -import kafka.utils._ import kafka.metrics.KafkaMetricsReporter -import kafka.consumer.{Blacklist,Whitelist,ConsumerConfig,Consumer} +import kafka.utils._ +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.common.utils.Utils + +import scala.collection.JavaConversions._ /** - * Consumer that dumps messages out to standard out. - * + * Consumer that dumps messages to standard out. */ object ConsoleConsumer extends Logging { + var messageCount = 0 + def main(args: Array[String]) { + val conf = new ConsumerConfig(args) + try { + run(conf) + } catch { + case e: Throwable => + error("Unknown error when running consumer: ", e) + System.exit(1); + } + } + + def run(conf: ConsumerConfig) { + + val consumer = + if (conf.useNewConsumer) { + new NewShinyConsumer(conf.topicArg, getNewConsumerProps(conf)) + } else { + checkZk(conf) + new OldConsumer(conf.filterSpec, getOldConsumerProps(conf)) + } + + addShutdownHook(consumer, conf) + + try { + process(conf.maxMessages, conf.formatter, consumer, conf.skipMessageOnError) + } finally { + consumer.cleanup() + reportRecordCount() + + // if we generated a random group id (as none specified explicitly) then avoid polluting zookeeper with persistent group data, this is a hack + if (!conf.groupIdPassed) + ZkUtils.maybeDeletePath(conf.options.valueOf(conf.zkConnectOpt), "/consumers/" + conf.consumerProps.get("group.id")) + } + } + + def checkZk(config: ConsumerConfig) { + if (!checkZkPathExists(config.options.valueOf(config.zkConnectOpt), "/brokers/ids")) { + System.err.println("No brokers found in ZK.") + System.exit(1) + } + + if (!config.options.has(config.deleteConsumerOffsetsOpt) && config.options.has(config.resetBeginningOpt) && + checkZkPathExists(config.options.valueOf(config.zkConnectOpt), "/consumers/" + config.consumerProps.getProperty("group.id") + "/offsets")) { + System.err.println("Found previous offset information for this group " + config.consumerProps.getProperty("group.id") + + ". Please use --delete-consumer-offsets to delete previous offsets metadata") + System.exit(1) + } + } + + def addShutdownHook(consumer: BaseConsumer, conf: ConsumerConfig) { + Runtime.getRuntime.addShutdownHook(new Thread() { + override def run() { + consumer.stop() + } + }) + } + + def process(maxMessages: Integer, formatter: MessageFormatter, consumer: BaseConsumer, skipMessageOnError: Boolean) { + while (messageCount < maxMessages || maxMessages == -1) { + messageCount += 1 + val msg: BaseConsumerRecord = try { + consumer.receive() + } catch { + case e: Throwable => { + error("Error processing message, stopping consumer: ", e) + consumer.stop() + return + } + } + try { + formatter.writeTo(msg.key, msg.value, System.out) + } catch { + case e: Throwable => + if (skipMessageOnError) { + error("Error processing message, skipping this message: ", e) + } else { + consumer.stop() + throw e + } + } + checkErr(formatter) + } + } + + def reportRecordCount() { + System.err.println(s"Processed a total of $messageCount messages") + } + + def checkErr(formatter: MessageFormatter) { + if (System.out.checkError()) { + // This means no one is listening to our output stream any more, time to shutdown + System.err.println("Unable to write to standard out, closing consumer.") + formatter.close() + System.exit(1) + } + } + + def getOldConsumerProps(config: ConsumerConfig): Properties = { + val props = new Properties + + props.putAll(config.consumerProps) + props.put("auto.offset.reset", if (config.fromBeginning) "smallest" else "largest") + props.put("zookeeper.connect", config.zkConnectionStr) + + if (!config.options.has(config.deleteConsumerOffsetsOpt) && config.options.has(config.resetBeginningOpt) && + checkZkPathExists(config.options.valueOf(config.zkConnectOpt), "/consumers/" + props.getProperty("group.id") + "/offsets")) { + System.err.println("Found previous offset information for this group " + props.getProperty("group.id") + + ". Please use --delete-consumer-offsets to delete previous offsets metadata") + System.exit(1) + } + + if (config.options.has(config.deleteConsumerOffsetsOpt)) + ZkUtils.maybeDeletePath(config.options.valueOf(config.zkConnectOpt), "/consumers/" + config.consumerProps.getProperty("group.id")) + + props + } + + def getNewConsumerProps(config: ConsumerConfig): Properties = { + val props = new Properties + + props.putAll(config.consumerProps) + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, if (config.options.has(config.resetBeginningOpt)) "earliest" else "latest") + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, config.bootstrapServer) + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, if (config.keyDeserializer != null) config.keyDeserializer else "org.apache.kafka.common.serialization.StringDeserializer") + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, if (config.valueDeserializer != null) config.valueDeserializer else "org.apache.kafka.common.serialization.ByteArrayDeserializer") + + props + } + + class ConsumerConfig(args: Array[String]) { val parser = new OptionParser val topicIdOpt = parser.accepts("topic", "The topic id to consume on.") - .withRequiredArg - .describedAs("topic") - .ofType(classOf[String]) + .withRequiredArg + .describedAs("topic") + .ofType(classOf[String]) val whitelistOpt = parser.accepts("whitelist", "Whitelist of topics to include for consumption.") - .withRequiredArg - .describedAs("whitelist") - .ofType(classOf[String]) + .withRequiredArg + .describedAs("whitelist") + .ofType(classOf[String]) val blacklistOpt = parser.accepts("blacklist", "Blacklist of topics to exclude from consumption.") - .withRequiredArg - .describedAs("blacklist") - .ofType(classOf[String]) + .withRequiredArg + .describedAs("blacklist") + .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]) - + "Multiple URLS can be given to allow fail-over.") + .withRequiredArg + .describedAs("urls") + .ofType(classOf[String]) val consumerConfigOpt = parser.accepts("consumer.config", "Consumer config properties file.") - .withRequiredArg - .describedAs("config file") - .ofType(classOf[String]) + .withRequiredArg + .describedAs("config file") + .ofType(classOf[String]) val messageFormatterOpt = parser.accepts("formatter", "The name of a class to use for formatting kafka messages for display.") - .withRequiredArg - .describedAs("class") - .ofType(classOf[String]) - .defaultsTo(classOf[DefaultMessageFormatter].getName) + .withRequiredArg + .describedAs("class") + .ofType(classOf[String]) + .defaultsTo(classOf[DefaultMessageFormatter].getName) val messageFormatterArgOpt = parser.accepts("property") - .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"); + .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.") + "start with the earliest message present in the log rather than the latest message.") 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") - .ofType(classOf[java.lang.Integer]) + .withRequiredArg + .describedAs("num_messages") + .ofType(classOf[java.lang.Integer]) val skipMessageOnErrorOpt = parser.accepts("skip-message-on-error", "If there is an error when processing a message, " + - "skip it instead of halt.") + "skip it instead of halt.") 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") + .describedAs("metrics directory") .ofType(classOf[java.lang.String]) + val useNewConsumerOpt = parser.accepts("new-consumer", "Use the new consumer implementation.") + val bootstrapServerOpt = parser.accepts("bootstrap-server") + .withRequiredArg + .describedAs("server to connect to") + .ofType(classOf[String]) + val keyDeserializerOpt = parser.accepts("key-deserializer") + .withRequiredArg + .describedAs("deserializer for key") + .ofType(classOf[String]) + val valueDeserializerOpt = parser.accepts("value-deserializer") + .withRequiredArg + .describedAs("deserializer for values") + .ofType(classOf[String]) - if(args.length == 0) + 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 useNewConsumer = options.has(useNewConsumerOpt) + val filterOpt = List(whitelistOpt, blacklistOpt).filter(options.has) val topicOrFilterOpt = List(topicIdOpt, whitelistOpt, blacklistOpt).filter(options.has) - 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) + val filterSpec = if (options.has(blacklistOpt)) new Blacklist(topicArg) else new Whitelist(topicArg) + val consumerProps = if (options.has(consumerConfigOpt)) + Utils.loadProps(options.valueOf(consumerConfigOpt)) else - new Whitelist(topicArg) + new Properties() + val zkConnectionStr = options.valueOf(zkConnectOpt) + val fromBeginning = options.has(resetBeginningOpt) + 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 bootstrapServer = options.valueOf(bootstrapServerOpt) + val keyDeserializer = options.valueOf(keyDeserializerOpt) + val valueDeserializer = options.valueOf(valueDeserializerOpt) + val formatter: MessageFormatter = messageFormatterClass.newInstance().asInstanceOf[MessageFormatter] + formatter.init(formatterArgs) + + CommandLineUtils.checkRequiredArgs(parser, options, if (useNewConsumer) bootstrapServerOpt else zkConnectOpt) - val csvMetricsReporterEnabled = options.has(csvMetricsReporterEnabledOpt) - if (csvMetricsReporterEnabled) { + if (!useNewConsumer && topicOrFilterOpt.size != 1) + CommandLineUtils.printUsageAndDie(parser, "Exactly one of whitelist/blacklist/topic is required.") + + if (options.has(csvMetricsReporterEnabledOpt)) { val csvReporterProps = new Properties() csvReporterProps.put("kafka.metrics.polling.interval.secs", "5") csvReporterProps.put("kafka.metrics.reporters", "kafka.metrics.KafkaCSVMetricsReporter") @@ -113,103 +270,26 @@ object ConsoleConsumer extends Logging { KafkaMetricsReporter.startReporters(verifiableProps) } - - - val consumerProps = if (options.has(consumerConfigOpt)) - Utils.loadProps(options.valueOf(consumerConfigOpt)) - else - new Properties() - - if(!consumerProps.containsKey("group.id")) { - consumerProps.put("group.id","console-consumer-" + new Random().nextInt(100000)) + //Provide the consumer with a randomly assigned group id + if(!consumerProps.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) { + consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG,"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)) - if(!consumerProps.containsKey("dual.commit.enabled")) - consumerProps.put("dual.commit.enabled","false") - if(!consumerProps.containsKey("offsets.storage")) - consumerProps.put("offsets.storage","zookeeper") - - 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) - } - - if(options.has(deleteConsumerOffsetsOpt)) - ZkUtils.maybeDeletePath(options.valueOf(zkConnectOpt), "/consumers/" + consumerProps.getProperty("group.id")) - - 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(!groupIdPassed) - ZkUtils.maybeDeletePath(options.valueOf(zkConnectOpt), "/consumers/" + consumerProps.get("group.id")) - } - }) - - var numMessages = 0L - val formatter: MessageFormatter = messageFormatterClass.newInstance().asInstanceOf[MessageFormatter] - formatter.init(formatterArgs) - try { - val stream = connector.createMessageStreamsByFilter(filterSpec, 1, new DefaultDecoder(), new DefaultDecoder()).get(0) - val iter = if(maxMessages >= 0) - stream.slice(0, maxMessages) - else - stream - - for(messageAndTopic <- iter) { - try { - formatter.writeTo(messageAndTopic.key, messageAndTopic.message, System.out) - numMessages += 1 - } catch { - case e: Throwable => - if (skipMessageOnError) - error("Error processing message, skipping this message: ", e) - else - throw e - } - if(System.out.checkError()) { - // This means no one is listening to our output stream any more, time to shutdown - System.err.println("Unable to write to standard out, closing consumer.") - System.err.println("Consumed %d messages".format(numMessages)) - formatter.close() - connector.shutdown() - System.exit(1) - } - } - } catch { - case e: Throwable => error("Error processing message, stopping consumer: ", e) - } - System.err.println("Consumed %d messages".format(numMessages)) - System.out.flush() - formatter.close() - connector.shutdown() - } - def tryParse(parser: OptionParser, args: Array[String]) = { - try { - parser.parse(args : _*) - } catch { - case e: OptionException => { - Utils.croak(e.getMessage) - null + def tryParse(parser: OptionParser, args: Array[String]) = { + try + parser.parse(args: _*) + catch { + case e: OptionException => + Utils.croak(e.getMessage) + null } } } def checkZkPathExists(zkUrl: String, path: String): Boolean = { try { - val zk = new ZkClient(zkUrl, 30*1000,30*1000, ZKStringSerializer); + val zk = ZkUtils.createZkClient(zkUrl, 30 * 1000, 30 * 1000) zk.exists(path) } catch { case _: Throwable => false @@ -219,7 +299,9 @@ object ConsoleConsumer extends Logging { trait MessageFormatter { def writeTo(key: Array[Byte], value: Array[Byte], output: PrintStream) + def init(props: Properties) {} + def close() {} } @@ -229,16 +311,16 @@ class DefaultMessageFormatter extends MessageFormatter { var lineSeparator = "\n".getBytes override def init(props: Properties) { - if(props.containsKey("print.key")) + if (props.containsKey("print.key")) printKey = props.getProperty("print.key").trim.toLowerCase.equals("true") - if(props.containsKey("key.separator")) + if (props.containsKey("key.separator")) keySeparator = props.getProperty("key.separator").getBytes - if(props.containsKey("line.separator")) + if (props.containsKey("line.separator")) lineSeparator = props.getProperty("line.separator").getBytes } def writeTo(key: Array[Byte], value: Array[Byte], output: PrintStream) { - if(printKey) { + if (printKey) { output.write(if (key == null) "null".getBytes() else key) output.write(keySeparator) } @@ -249,6 +331,7 @@ class DefaultMessageFormatter extends MessageFormatter { class NoOpMessageFormatter extends MessageFormatter { override def init(props: Properties) {} + def writeTo(key: Array[Byte], value: Array[Byte], output: PrintStream) {} } diff --git a/core/src/main/scala/kafka/tools/ConsoleProducer.scala b/core/src/main/scala/kafka/tools/ConsoleProducer.scala index 8e9ba0b284671..de50f651624c4 100644 --- a/core/src/main/scala/kafka/tools/ConsoleProducer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleProducer.scala @@ -27,59 +27,22 @@ 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] - val props = new Properties - props.put("topic", config.topic) - props.putAll(config.cmdLineProps) - reader.init(System.in, props) - try { + val config = new ProducerConfig(args) + val reader = Class.forName(config.readerClass).newInstance().asInstanceOf[MessageReader] + reader.init(System.in, getReaderProps(config)) + val producer = if(config.useNewProducer) { - import org.apache.kafka.clients.producer.ProducerConfig - - 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") - - new NewShinyProducer(props) + new NewShinyProducer(getNewProducerProps(config)) } else { - 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") - - new OldProducer(props) + new OldProducer(getOldProducerProps(config)) } Runtime.getRuntime.addShutdownHook(new Thread() { @@ -95,6 +58,9 @@ object ConsoleProducer { producer.send(message.topic, message.key, message.message) } while(message != null) } catch { + case e: joptsimple.OptionException => + System.err.println(e.getMessage) + System.exit(1) case e: Exception => e.printStackTrace System.exit(1) @@ -102,6 +68,66 @@ object ConsoleProducer { 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.") @@ -113,8 +139,8 @@ object ConsoleProducer { .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 'gzip' or 'snappy'." + - "If specified without value, than it defaults to 'gzip'") + 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]) @@ -132,13 +158,13 @@ object ConsoleProducer { .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 suffient batch size. The value is given in ms.") + " 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 suffient batch size.") + " messages will queue awaiting sufficient batch size.") .withRequiredArg .describedAs("queue_size") .ofType(classOf[java.lang.Integer]) @@ -209,6 +235,10 @@ object ConsoleProducer { .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 : _*) @@ -241,6 +271,7 @@ object ConsoleProducer { 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) diff --git a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala index d1e7c434e7785..c39fbfe483b82 100644 --- a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala +++ b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala @@ -24,11 +24,11 @@ import kafka.utils._ import kafka.consumer.SimpleConsumer 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 scala.Some import org.I0Itec.zkclient.exception.ZkNoNodeException object ConsumerOffsetChecker extends Logging { @@ -107,6 +107,8 @@ object ConsumerOffsetChecker extends Logging { } def main(args: Array[String]) { + warn("WARNING: ConsumerOffsetChecker is deprecated and will be dropped in releases following 0.9.0. Use ConsumerGroupCommand instead.") + val parser = new OptionParser() val zkConnectOpt = parser.accepts("zookeeper", "ZooKeeper connect string."). @@ -149,7 +151,7 @@ object ConsumerOffsetChecker extends Logging { 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 @@ -162,7 +164,7 @@ object ConsumerOffsetChecker extends Logging { 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().buffer) + val offsetFetchResponse = OffsetFetchResponse.readFrom(channel.receive().payload()) debug("Received offset fetch response %s.".format(offsetFetchResponse)) offsetFetchResponse.requestInfo.foreach { case (topicAndPartition, offsetAndMetadata) => diff --git a/core/src/main/scala/kafka/tools/ConsumerPerformance.scala b/core/src/main/scala/kafka/tools/ConsumerPerformance.scala index 093c800ea7f8a..c7f907238b152 100644 --- a/core/src/main/scala/kafka/tools/ConsumerPerformance.scala +++ b/core/src/main/scala/kafka/tools/ConsumerPerformance.scala @@ -17,15 +17,25 @@ package kafka.tools -import java.util.concurrent.CountDownLatch +import java.util + +import org.apache.kafka.common.TopicPartition + +import scala.collection.JavaConversions._ import java.util.concurrent.atomic.AtomicLong import java.nio.channels.ClosedByInterruptException import org.apache.log4j.Logger -import kafka.message.Message -import kafka.utils.{ZkUtils, CommandLineUtils} +import org.apache.kafka.clients.consumer.{ConsumerRebalanceListener, KafkaConsumer} +import org.apache.kafka.common.serialization.ByteArrayDeserializer +import org.apache.kafka.common.utils.Utils +import kafka.utils.CommandLineUtils import java.util.{ Random, Properties } -import kafka.consumer._ +import kafka.consumer.Consumer +import kafka.consumer.ConsumerConnector +import kafka.consumer.KafkaStream +import kafka.consumer.ConsumerTimeoutException import java.text.SimpleDateFormat +import java.util.concurrent.atomic.AtomicBoolean /** * Performance test for the full zookeeper consumer @@ -37,55 +47,131 @@ object ConsumerPerformance { val config = new ConsumerPerfConfig(args) logger.info("Starting consumer...") - var totalMessagesRead = new AtomicLong(0) - var totalBytesRead = new AtomicLong(0) + val totalMessagesRead = new AtomicLong(0) + val totalBytesRead = new AtomicLong(0) + val consumerTimeout = new AtomicBoolean(false) 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") + println("start.time, end.time, 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") + println("time, 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) + var startMs, endMs = 0L + if(config.useNewConsumer) { + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](config.props) + consumer.subscribe(List(config.topic)) + startMs = System.currentTimeMillis + consume(consumer, List(config.topic), 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, consumerTimeout) - val consumerConnector: ConsumerConnector = Consumer.create(config.consumerConfig) + 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 + if(consumerTimeout.get()) + endMs = System.currentTimeMillis - consumerConfig.consumerTimeoutMs + else + endMs = System.currentTimeMillis + 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]], topics: List[String], count: Long, timeout: Long, config: ConsumerPerfConfig, totalMessagesRead: AtomicLong, totalBytesRead: AtomicLong) { + var bytesRead = 0L + var messagesRead = 0L + var lastBytesRead = 0L + var lastMessagesRead = 0L - 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) + // Wait for group join, metadata fetch, etc + val joinTimeout = 10000 + val isAssigned = new AtomicBoolean(false) + consumer.subscribe(topics, new ConsumerRebalanceListener { + def onPartitionsAssigned(partitions: util.Collection[TopicPartition]) { + isAssigned.set(true) + } + def onPartitionsRevoked(partitions: util.Collection[TopicPartition]) { + isAssigned.set(false) + }}) + val joinStart = System.currentTimeMillis() + while (!isAssigned.get()) { + if (System.currentTimeMillis() - joinStart >= joinTimeout) { + throw new Exception("Timed out waiting for initial group join.") + } + consumer.poll(100) + } + consumer.seekToBeginning() - logger.info("Sleeping for 1 second.") - Thread.sleep(1000) - logger.info("starting threads") + // Now start the benchmark val startMs = System.currentTimeMillis - for (thread <- threadList) - thread.start - - for (thread <- threadList) - thread.join - - 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)) + var lastReportTime: Long = startMs + var lastConsumedTime = System.currentTimeMillis + + while(messagesRead < count && System.currentTimeMillis() - lastConsumedTime <= timeout) { + val records = consumer.poll(100) + if(records.count() > 0) + lastConsumedTime = 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 + } + } } - System.exit(0) + + 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", "REQUIRED: The connection string for the zookeeper connection in the form host:port. " + - "Multiple URLS can be given to allow fail-over.") + 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") @@ -117,20 +203,42 @@ object ConsumerPerformance { .describedAs("count") .ofType(classOf[java.lang.Integer]) .defaultsTo(1) + val useNewConsumerOpt = parser.accepts("new-consumer", "Use the new consumer implementation.") + val consumerConfigOpt = parser.accepts("consumer.config", "Consumer config properties file.") + .withRequiredArg + .describedAs("config file") + .ofType(classOf[String]) val options = parser.parse(args: _*) - CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, zkConnectOpt) - - 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) + CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, numMessagesOpt) + + val useNewConsumer = options.has(useNewConsumerOpt) + + val props = if (options.has(consumerConfigOpt)) + Utils.loadProps(options.valueOf(consumerConfigOpt)) + else + 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 @@ -141,7 +249,7 @@ object ConsumerPerformance { } class ConsumerPerfThread(threadId: Int, name: String, stream: KafkaStream[Array[Byte], Array[Byte]], - config: ConsumerPerfConfig, totalMessagesRead: AtomicLong, totalBytesRead: AtomicLong) + config: ConsumerPerfConfig, totalMessagesRead: AtomicLong, totalBytesRead: AtomicLong, consumerTimeout: AtomicBoolean) extends Thread(name) { override def run() { @@ -161,7 +269,7 @@ object ConsumerPerformance { if (messagesRead % config.reportingInterval == 0) { if (config.showDetailedStats) - printMessage(threadId, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, lastReportTime, System.currentTimeMillis) + printProgressMessage(threadId, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, lastReportTime, System.currentTimeMillis, config.dateFormat) lastReportTime = System.currentTimeMillis lastMessagesRead = messagesRead lastBytesRead = bytesRead @@ -170,24 +278,16 @@ object ConsumerPerformance { } catch { case _: InterruptedException => case _: ClosedByInterruptException => - case _: ConsumerTimeoutException => + case _: ConsumerTimeoutException => { + consumerTimeout.set(true); + } case e: Throwable => e.printStackTrace() } totalMessagesRead.addAndGet(messagesRead) totalBytesRead.addAndGet(bytesRead) if (config.showDetailedStats) - printMessage(threadId, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, System.currentTimeMillis) + printProgressMessage(threadId, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, System.currentTimeMillis, config.dateFormat) } - 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)) - } } - } 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 8e9d47b8d4adc..fc11a2ab220a9 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -25,6 +25,7 @@ import collection.mutable import joptsimple.OptionParser import kafka.serializer.Decoder import kafka.utils.VerifiableProperties +import org.apache.kafka.common.utils.Utils object DumpLogSegments { @@ -64,8 +65,8 @@ object DumpLogSegments { val maxMessageSize = options.valueOf(maxMessageSizeOpt).intValue() val isDeepIteration = if(options.has(deepIterationOpt)) true else false - val valueDecoder: Decoder[_] = Utils.createObject[Decoder[_]](options.valueOf(valueDecoderOpt), new VerifiableProperties) - val keyDecoder: Decoder[_] = Utils.createObject[Decoder[_]](options.valueOf(keyDecoderOpt), new VerifiableProperties) + 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)]] @@ -104,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) { @@ -181,7 +181,7 @@ object DumpLogSegments { case NoCompressionCodec => getSingleMessageIterator(messageAndOffset) case _ => - ByteBufferMessageSet.decompress(message).iterator + ByteBufferMessageSet.deepIterator(message) } } else getSingleMessageIterator(messageAndOffset) diff --git a/core/src/main/scala/kafka/tools/EndToEndLatency.scala b/core/src/main/scala/kafka/tools/EndToEndLatency.scala new file mode 100755 index 0000000000000..8197dfc03b6d1 --- /dev/null +++ b/core/src/main/scala/kafka/tools/EndToEndLatency.scala @@ -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. + */ + +package kafka.tools + +import java.util.{Arrays, Properties} + +import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} +import org.apache.kafka.clients.producer._ +import org.apache.kafka.common.utils.Utils + +import scala.collection.JavaConversions._ + + +/** + * This class records the average end to end latency for a single message to travel through Kafka + * + * broker_list = location of the bootstrap broker for both the producer and the consumer + * num_messages = # messages to send + * producer_acks = See ProducerConfig.ACKS_DOC + * message_size_bytes = size of each message in bytes + * + * e.g. [localhost:9092 test 10000 1 20] + */ + +object EndToEndLatency { + private val timeout: Long = 60000 + + def main(args: Array[String]) { + println(args.length) + if (args.length != 5 && args.length != 6) { + System.err.println("USAGE: java " + getClass.getName + " broker_list topic num_messages producer_acks message_size_bytes [optional] ssl_properties_file") + System.exit(1) + } + + val brokerList = args(0) + val topic = args(1) + val numMessages = args(2).toInt + val producerAcks = args(3) + val messageLen = args(4).toInt + val sslPropsFile = if (args.length == 6) args(5) else "" + + if (!List("1", "all").contains(producerAcks)) + throw new IllegalArgumentException("Latency testing requires synchronous acknowledgement. Please use 1 or all") + + val consumerProps = if (sslPropsFile.equals("")) new Properties() else Utils.loadProps(sslPropsFile) + consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, "test-group-" + System.currentTimeMillis()) + consumerProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") + consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest") + 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") + consumerProps.put(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG, "0") //ensure we have no temporal batching + + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](consumerProps) + consumer.subscribe(List(topic)) + + val producerProps = if (sslPropsFile.equals("")) new Properties() else Utils.loadProps(sslPropsFile) + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + producerProps.put(ProducerConfig.LINGER_MS_CONFIG, "0") //ensure writes are synchronous + 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) + + def finalise() { + consumer.commitSync() + producer.close() + consumer.close() + } + + //Ensure we are at latest offset. seekToEnd evaluates lazily, that is to say actually performs the seek only when + //a poll() or position() request is issued. Hence we need to poll after we seek to ensure we see our first write. + consumer.seekToEnd() + consumer.poll(0) + + var totalTime = 0.0 + val latencies = new Array[Long](numMessages) + + for (i <- 0 until numMessages) { + val message = randomBytesOfLen(messageLen) + val begin = System.nanoTime + + //Send message (of random bytes) synchronously then immediately poll for it + producer.send(new ProducerRecord[Array[Byte], Array[Byte]](topic, message)).get() + val recordIter = consumer.poll(timeout).iterator + + val elapsed = System.nanoTime - begin + + //Check we got results + if (!recordIter.hasNext) { + finalise() + throw new RuntimeException(s"poll() timed out before finding a result (timeout:[$timeout])") + } + + //Check result matches the original record + val sent = new String(message) + val read = new String(recordIter.next().value()) + if (!read.equals(sent)) { + finalise() + throw new RuntimeException(s"The message read [$read] did not match the message sent [$sent]") + } + + //Check we only got the one message + if (recordIter.hasNext) { + var count = 1 + for (elem <- recordIter) count += 1 + throw new RuntimeException(s"Only one result was expected during this test. We found [$count]") + } + + //Report progress + if (i % 1000 == 0) + println(i + "\t" + elapsed / 1000.0 / 1000.0) + totalTime += elapsed + latencies(i) = elapsed / 1000 / 1000 + } + + //Results + 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)) + + finalise() + } + + def randomBytesOfLen(len: Int): Array[Byte] = { + Array.fill(len)((scala.util.Random.nextInt(26) + 65).toByte) + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/tools/ExportZkOffsets.scala b/core/src/main/scala/kafka/tools/ExportZkOffsets.scala index 4d051bc2db12f..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, CommandLineUtils} +import kafka.utils.{Logging, ZkUtils, ZKGroupTopicDirs, CommandLineUtils} import org.I0Itec.zkclient.ZkClient @@ -76,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 @@ -114,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/ImportZkOffsets.scala b/core/src/main/scala/kafka/tools/ImportZkOffsets.scala index abe09721b13f7..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, CommandLineUtils} +import kafka.utils.{Logging, ZkUtils, CommandLineUtils} import org.I0Itec.zkclient.ZkClient @@ -68,7 +68,7 @@ object ImportZkOffsets extends Logging { 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) @@ -89,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 1d1a120c45ff7..d335b3e65703b 100644 --- a/core/src/main/scala/kafka/tools/JmxTool.scala +++ b/core/src/main/scala/kafka/tools/JmxTool.scala @@ -97,13 +97,14 @@ object JmxTool extends Logging { val numExpectedAttributes: Map[ObjectName, Int] = attributesWhitelistExists match { case true => queries.map((_, attributesWhitelist.get.size)).toMap - case false => names.map((name: ObjectName) => - (name, mbsc.getMBeanInfo(name).getAttributes().map(_.getName).size)).toMap + case false => names.map{(name: ObjectName) => + val mbean = mbsc.getMBeanInfo(name) + (name, mbsc.getAttributes(name, mbean.getAttributes.map(_.getName)).size)}.toMap } // 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) { @@ -113,7 +114,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) @@ -137,4 +138,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 b8698ee1469c8..fbe0c834b6462 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -17,37 +17,71 @@ package kafka.tools -import kafka.utils.{SystemTime, Utils, CommandLineUtils, Logging} -import kafka.consumer._ -import kafka.serializer._ -import kafka.producer.{OldProducer, NewShinyProducer, BaseProducer} -import kafka.metrics.KafkaMetricsGroup +import java.util +import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} +import java.util.concurrent.{CountDownLatch, TimeUnit} +import java.util.{Collections, Properties} -import org.apache.kafka.clients.producer.ProducerRecord +import com.yammer.metrics.core.Gauge +import joptsimple.OptionParser +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, CoreUtils, Logging} +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 joptsimple.OptionParser -import java.util.Random -import java.util.concurrent.atomic.AtomicInteger -import java.util.concurrent.{TimeUnit, BlockingQueue, ArrayBlockingQueue, CountDownLatch} -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[ConsumerThread] = null - private var producerThreads: Seq[ProducerThread] = null - - private val shutdownMessage : ProducerRecord = new ProducerRecord("shutdown", "shutdown".getBytes) + 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]) @@ -58,16 +92,6 @@ object MirrorMaker extends Logging { .describedAs("config file") .ofType(classOf[String]) - val useNewProducerOpt = parser.accepts("new.producer", - "Use the new producer implementation.") - - 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() @@ -75,13 +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() @@ -89,17 +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.") - - if(args.length == 0) + + if (args.length == 0) CommandLineUtils.printUsageAndDie(parser, "Continuously copy data between two Kafka clusters.") - val options = parser.parse(args : _*) + + val options = parser.parse(args: _*) if (options.has(helpOpt)) { parser.printHelpOn(System.out) @@ -112,216 +168,285 @@ object MirrorMaker extends Logging { System.exit(1) } - val numProducers = options.valueOf(numProducersOpt).intValue() + abortOnSendFailure = options.valueOf(abortOnSendFailureOpt).toBoolean + offsetCommitIntervalMs = options.valueOf(offsetCommitIntervalMsOpt).intValue() val numStreams = options.valueOf(numStreamsOpt).intValue() - val bufferSize = options.valueOf(bufferSizeOpt).intValue() - - // create consumer streams - connectors = options.valuesOf(consumerConfigOpt).toList - .map(cfg => new ConsumerConfig(Utils.loadProps(cfg))) - .map(new ZookeeperConsumerConnector(_)) - val numConsumers = connectors.size * numStreams - - // create a data channel btw the consumers and the producers - val mirrorDataChannel = new DataChannel(bufferSize, numConsumers, numProducers) - // create producer threads - val useNewProducer = options.has(useNewProducerOpt) - val producerProps = Utils.loadProps(options.valueOf(producerConfigOpt)) - val clientId = producerProps.getProperty("client.id", "") - producerThreads = (0 until numProducers).map(i => { - producerProps.setProperty("client.id", clientId + "-" + i) - val producer = - if (useNewProducer) - new NewShinyProducer(producerProps) - else - new OldProducer(producerProps) - new ProducerThread(mirrorDataChannel, producer, i) + Runtime.getRuntime.addShutdownHook(new Thread("MirrorMakerShutdownHook") { + override def run() { + cleanShutdown() + } }) + + // 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") + // Always set producer key and value serializer to ByteArraySerializer. + producerProps.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + producerProps.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + 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) + } - // create consumer threads + // 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, new DefaultDecoder(), new DefaultDecoder())).flatten - } catch { - case t: Throwable => - fatal("Unable to create stream - shutting down mirror maker.") - connectors.foreach(_.shutdown) - } - consumerThreads = streams.zipWithIndex.map(streamAndIndex => new ConsumerThread(streamAndIndex._1, mirrorDataChannel, streamAndIndex._2)) - assert(consumerThreads.size == numConsumers) - - 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 } - }) - - consumerThreads.foreach(_.start) - producerThreads.foreach(_.start) + } - // we wait on producer's shutdown latch instead of consumers - // since the consumer threads can hit a timeout/other exception; - // but in this case the producer should still be able to shutdown - // based on the shutdown message in the channel - producerThreads.foreach(_.awaitShutdown) + mirrorMakerThreads.foreach(_.start()) + mirrorMakerThreads.foreach(_.awaitShutdown()) } - def cleanShutdown() { - if (connectors != null) connectors.foreach(_.shutdown) - if (consumerThreads != null) consumerThreads.foreach(_.awaitShutdown) - if (producerThreads != null) { - producerThreads.foreach(_.shutdown) - producerThreads.foreach(_.awaitShutdown) + def commitOffsets(connector: ZookeeperConsumerConnector) { + if (!exitingOnSendFailure) { + trace("Committing offsets.") + connector.commitOffsets + } else { + info("Exiting on send failure, skip committing offsets.") } - info("Kafka mirror maker shutdown successfully") } - class DataChannel(capacity: Int, numProducers: Int, numConsumers: Int) extends KafkaMetricsGroup { - - val queues = new Array[BlockingQueue[ProducerRecord]](numConsumers) - for (i <- 0 until numConsumers) - queues(i) = new ArrayBlockingQueue[ProducerRecord](capacity) - - private val counter = new AtomicInteger(new Random().nextInt()) - - // We use a single meter for aggregated wait percentage for the data channel. - // Since meter is calculated as total_recorded_value / time_window and - // time_window is independent of the number of threads, each recorded wait - // time should be discounted by # threads. - private val waitPut = newMeter("MirrorMaker-DataChannel-WaitOnPut", "percent", TimeUnit.NANOSECONDS) - private val waitTake = newMeter("MirrorMaker-DataChannel-WaitOnTake", "percent", TimeUnit.NANOSECONDS) - private val channelSizeHist = newHistogram("MirrorMaker-DataChannel-Size") - - def put(record: ProducerRecord) { - // If the key of the message is empty, use round-robin to select the queue - // Otherwise use the queue based on the key value so that same key-ed messages go to the same queue - val queueId = - if(record.key() != null) { - Utils.abs(java.util.Arrays.hashCode(record.key())) % numConsumers - } else { - Utils.abs(counter.getAndIncrement()) % numConsumers - } - val queue = queues(queueId) - - var putSucceed = false - while (!putSucceed) { - val startPutTime = SystemTime.nanoseconds - putSucceed = queue.offer(record, 500, TimeUnit.MILLISECONDS) - waitPut.mark((SystemTime.nanoseconds - startPutTime) / numProducers) - } - channelSizeHist.update(queue.size) - } - - def take(queueId: Int): ProducerRecord = { - val queue = queues(queueId) - var data: ProducerRecord = null - while (data == null) { - val startTakeTime = SystemTime.nanoseconds - data = queue.poll(500, TimeUnit.MILLISECONDS) - waitTake.mark((SystemTime.nanoseconds - startTakeTime) / numConsumers) + def cleanShutdown() { + 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()) } - channelSizeHist.update(queue.size) - data + info("Closing producer.") + producer.close() + info("Kafka mirror maker shutdown successfully") } } - class ConsumerThread(stream: KafkaStream[Array[Byte], Array[Byte]], - mirrorDataChannel: DataChannel, - threadId: Int) - extends Thread with Logging with KafkaMetricsGroup { + 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-consumer-" + 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 consumer thread " + threadName) + info("Starting mirror maker thread " + threadName) try { - for (msgAndMetadata <- stream) { - val data = new ProducerRecord(msgAndMetadata.topic, msgAndMetadata.key, msgAndMetadata.message) - mirrorDataChannel.put(data) + // 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("Consumer thread stopped") + 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("Consumer thread shutdown complete") + info("Mirror maker thread shutdown complete") } catch { - case e: InterruptedException => fatal("Shutdown of the consumer thread interrupted. This might leak data!") + case ie: InterruptedException => + warn("Shutdown of the mirror maker thread interrupted") } } } - class ProducerThread (val dataChannel: DataChannel, - val producer: BaseProducer, - val threadId: Int) extends Thread with Logging with KafkaMetricsGroup { - private val threadName = "mirrormaker-producer-" + threadId - private val shutdownComplete: CountDownLatch = new CountDownLatch(1) - this.logIdent = "[%s] ".format(threadName) + private class MirrorMakerProducer(val producerProps: Properties) { - setName(threadName) + val sync = producerProps.getProperty("producer.type", "async").equals("sync") - override def run { - info("Starting mirror maker producer thread " + threadName) - try { - while (true) { - val data: ProducerRecord = dataChannel.take(threadId) - trace("Sending message with value size %d".format(data.value().size)) - if(data eq shutdownMessage) { - info("Received shutdown message") - return - } - producer.send(data.topic(), data.key(), data.value()) - } - } catch { - case t: Throwable => { - fatal("Producer thread failure due to ", t) - } - } finally { - shutdownComplete.countDown - info("Producer thread stopped") + 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 shutdown { - try { - info("Producer thread " + threadName + " shutting down") - dataChannel.put(shutdownMessage) - } - catch { - case ie: InterruptedException => { - warn("Interrupt during shutdown of ProducerThread") - } - } + def flush() { + this.producer.flush() } - def awaitShutdown { - try { - shutdownComplete.await - producer.close - info("Producer thread shutdown complete") - } catch { - case ie: InterruptedException => { - warn("Shutdown of the producer thread interrupted") + 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/core/src/main/scala/kafka/tools/PerfConfig.scala b/core/src/main/scala/kafka/tools/PerfConfig.scala index 129cc013f68d2..298bb29fe797f 100644 --- a/core/src/main/scala/kafka/tools/PerfConfig.scala +++ b/core/src/main/scala/kafka/tools/PerfConfig.scala @@ -22,11 +22,10 @@ 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/core/src/main/scala/kafka/tools/ProducerPerformance.scala b/core/src/main/scala/kafka/tools/ProducerPerformance.scala index f61c7c701fd85..e299f8b26ecce 100644 --- a/core/src/main/scala/kafka/tools/ProducerPerformance.scala +++ b/core/src/main/scala/kafka/tools/ProducerPerformance.scala @@ -28,8 +28,8 @@ import java.util.concurrent.atomic.AtomicLong import java.util._ import java.text.SimpleDateFormat import java.math.BigInteger -import scala.collection.immutable.List +import org.apache.kafka.common.utils.Utils import org.apache.log4j.Logger /** @@ -38,7 +38,6 @@ import org.apache.log4j.Logger object ProducerPerformance extends Logging { def main(args: Array[String]) { - val logger = Logger.getLogger(getClass) val config = new ProducerPerfConfig(args) if (!config.isFixedSize) @@ -71,15 +70,19 @@ object ProducerPerformance extends Logging { } 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.") + 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 producerConfigOpt = parser.accepts("producer.config", "Producer config properties file.") + .withRequiredArg + .describedAs("config file") + .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") + val producerRequestTimeoutMsOpt = parser.accepts("request-timeout-ms", "The producer request timeout in ms") .withRequiredArg() .ofType(classOf[java.lang.Integer]) .defaultsTo(3000) @@ -116,9 +119,9 @@ object ProducerPerformance extends Logging { .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 outputted here") .withRequiredArg - .describedAs("metrics dictory") + .describedAs("metrics directory") .ofType(classOf[java.lang.String]) val useNewProducerOpt = parser.accepts("new-producer", "Use the new producer implementation.") @@ -151,6 +154,11 @@ object ProducerPerformance extends Logging { val csvMetricsReporterEnabled = options.has(csvMetricsReporterEnabledOpt) + val producerProps = if (options.has(producerConfigOpt)) + Utils.loadProps(options.valueOf(producerConfigOpt)) + else + new Properties() + if (csvMetricsReporterEnabled) { val props = new Properties() props.put("kafka.metrics.polling.interval.secs", "1") @@ -181,16 +189,19 @@ object ProducerPerformance extends Logging { val producer = if (config.useNewProducer) { import org.apache.kafka.clients.producer.ProducerConfig + props.putAll(config.producerProps) 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.putAll(config.producerProps) props.put("metadata.broker.list", config.brokerList) props.put("compression.codec", config.compressionCodec.codec.toString) props.put("send.buffer.bytes", (64 * 1024).toString) @@ -234,7 +245,7 @@ object ProducerPerformance extends Logging { val seqMsgString = String.format("%1$-" + msgSize + "s", msgHeader).replace(' ', 'x') debug(seqMsgString) - return seqMsgString.getBytes() + seqMsgString.getBytes() } private def generateProducerData(topic: String, messageId: Long): Array[Byte] = { diff --git a/core/src/main/scala/kafka/tools/ReplayLogProducer.scala b/core/src/main/scala/kafka/tools/ReplayLogProducer.scala index 3393a3dd574ac..2b8537b89fafa 100644 --- a/core/src/main/scala/kafka/tools/ReplayLogProducer.scala +++ b/core/src/main/scala/kafka/tools/ReplayLogProducer.scala @@ -120,11 +120,13 @@ object ReplayLogProducer extends Logging { 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 producer = new KafkaProducer(config.producerProps) + val producer = new KafkaProducer[Array[Byte],Array[Byte]](config.producerProps) override def run() { info("Starting consumer thread..") @@ -137,7 +139,7 @@ object ReplayLogProducer extends Logging { stream for (messageAndMetadata <- iter) { try { - val response = producer.send(new ProducerRecord(config.outputTopic, + val response = producer.send(new ProducerRecord[Array[Byte],Array[Byte]](config.outputTopic, messageAndMetadata.key(), messageAndMetadata.message())) if(config.isSync) { response.get() diff --git a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala index ba6ddd7a909df..fd9daeca615ac 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,7 @@ 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.") @@ -197,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] @@ -329,13 +329,13 @@ private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPa if (currentTimeMs - lastReportTime > reportInterval) { println(ReplicaVerificationTool.dateFormat.format(new Date(currentTimeMs)) + ": max lag is " + maxLag + " for partition " + maxLagTopicAndPartition + " at offset " + offsetWithMaxLag - + " among " + messageSetCache.size + " paritions") + + " among " + messageSetCache.size + " partitions") lastReportTime = currentTimeMs } } } -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/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala b/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala index 7602b8d705970..5e3c605248fd7 100644 --- a/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala +++ b/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala @@ -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) @@ -79,7 +81,7 @@ object SimpleConsumerPerformance { 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 @@ -141,7 +143,7 @@ object SimpleConsumerPerformance { val options = parser.parse(args : _*) - CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, urlOpt) + CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, urlOpt, numMessagesOpt) val url = new URI(options.valueOf(urlOpt)) val fetchSize = options.valueOf(fetchSizeOpt).intValue 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 b4f903b6c7c3b..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 @@ -142,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) } } @@ -167,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, @@ -188,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 d298e7e81acc7..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, CommandLineUtils} +import kafka.utils.{CoreUtils, Logging, CommandLineUtils} import kafka.common.Topic import java.io.{BufferedOutputStream, OutputStream} @@ -115,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/TestEndToEndLatency.scala b/core/src/main/scala/kafka/tools/TestEndToEndLatency.scala deleted file mode 100644 index 67196f30af1cf..0000000000000 --- a/core/src/main/scala/kafka/tools/TestEndToEndLatency.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.tools - -import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord, KafkaProducer} - -import kafka.consumer._ - -import java.util.Properties -import java.util.Arrays - -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) - val producer = new KafkaProducer(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(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/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 aef8361b73a09..db2721fec60cf 100644 --- a/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala +++ b/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala @@ -19,7 +19,7 @@ package kafka.tools import joptsimple.OptionParser import org.I0Itec.zkclient.ZkClient -import kafka.utils.{Logging, ZKGroupTopicDirs, ZkUtils, ZKStringSerializer, CommandLineUtils} +import kafka.utils.{Logging, ZKGroupTopicDirs, ZkUtils, CommandLineUtils} object VerifyConsumerRebalance extends Logging { def main(args: Array[String]) { @@ -48,7 +48,7 @@ object VerifyConsumerRebalance extends Logging { 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)) @@ -79,9 +79,7 @@ object VerifyConsumerRebalance extends Logging { 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))) @@ -95,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/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 086a62483fad0..c51735dd728b3 100644 --- a/core/src/main/scala/kafka/utils/CommandLineUtils.scala +++ b/core/src/main/scala/kafka/utils/CommandLineUtils.scala @@ -50,10 +50,10 @@ object CommandLineUtils extends Logging { /** * Print usage and exit */ - def printUsageAndDie(parser: OptionParser, message: String) { + def printUsageAndDie(parser: OptionParser, message: String): Nothing = { System.err.println(message) parser.printHelpOn(System.err) - System.exit(1) + sys.exit(1) } /** @@ -73,4 +73,4 @@ object CommandLineUtils extends Logging { } 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 9a16343d2ff71..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,34 +88,45 @@ 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 = Utils.runnable { - try { - trace("Begining execution of scheduled task '%s'.".format(name)) - fun() - } catch { - case t: Throwable => error("Uncaught exception in scheduled task '" + name +"'", t) - } finally { - trace("Completed execution of scheduled task '%s'.".format(name)) + this synchronized { + ensureStarted + val runnable = CoreUtils.runnable { + try { + trace("Begining execution of scheduled task '%s'.".format(name)) + fun() + } catch { + case t: Throwable => error("Uncaught exception in scheduled task '" + name +"'", t) + } finally { + trace("Completed execution of scheduled task '%s'.".format(name)) + } } + if(period >= 0) + executor.scheduleAtFixedRate(runnable, delay, period, unit) + else + executor.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/NetworkClientBlockingOps.scala b/core/src/main/scala/kafka/utils/NetworkClientBlockingOps.scala new file mode 100644 index 0000000000000..9ed9d29a2932e --- /dev/null +++ b/core/src/main/scala/kafka/utils/NetworkClientBlockingOps.scala @@ -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 kafka.utils + +import java.io.IOException +import org.apache.kafka.clients.{ClientRequest, ClientResponse, NetworkClient} +import org.apache.kafka.common.Node + +import scala.annotation.tailrec +import scala.collection.JavaConverters._ + +import org.apache.kafka.common.utils.{Time => JTime} + +object NetworkClientBlockingOps { + implicit def networkClientBlockingOps(client: NetworkClient): NetworkClientBlockingOps = + new NetworkClientBlockingOps(client) +} + +/** + * Provides extension methods for `NetworkClient` that are useful for implementing blocking behaviour. Use with care. + * + * Example usage: + * + * {{{ + * val networkClient: NetworkClient = ... + * import NetworkClientBlockingOps._ + * networkClient.blockingReady(...) + * }}} + */ +class NetworkClientBlockingOps(val client: NetworkClient) extends AnyVal { + + /** + * Invokes `client.ready` followed by 0 or more `client.poll` invocations until the connection to `node` is ready, + * the timeout expires or the connection fails. + * + * It returns `true` if the call completes normally or `false` if the timeout expires. If the connection fails, + * an `IOException` is thrown instead. + * + * This method is useful for implementing blocking behaviour on top of the non-blocking `NetworkClient`, use it with + * care. + */ + def blockingReady(node: Node, timeout: Long)(implicit time: JTime): Boolean = { + client.ready(node, time.milliseconds()) || pollUntil(timeout) { (_, now) => + if (client.isReady(node, now)) + true + else if (client.connectionFailed(node)) + throw new IOException(s"Connection to $node failed") + else false + } + } + + /** + * Invokes `client.send` followed by 1 or more `client.poll` invocations until a response is received, + * the timeout expires or a disconnection happens. + * + * It returns `true` if the call completes normally or `false` if the timeout expires. In the case of a disconnection, + * an `IOException` is thrown instead. + * + * This method is useful for implementing blocking behaviour on top of the non-blocking `NetworkClient`, use it with + * care. + */ + def blockingSendAndReceive(request: ClientRequest, timeout: Long)(implicit time: JTime): Option[ClientResponse] = { + client.send(request, time.milliseconds()) + + pollUntilFound(timeout) { case (responses, _) => + val response = responses.find { response => + response.request.request.header.correlationId == request.request.header.correlationId + } + response.foreach { r => + if (r.wasDisconnected) { + val destination = request.request.destination + throw new IOException(s"Connection to $destination was disconnected before the response was read") + } + } + response + } + + } + + /** + * Invokes `client.poll` until `predicate` returns `true` or the timeout expires. + * + * It returns `true` if the call completes normally or `false` if the timeout expires. Exceptions thrown via + * `predicate` are not handled and will bubble up. + * + * This method is useful for implementing blocking behaviour on top of the non-blocking `NetworkClient`, use it with + * care. + */ + private def pollUntil(timeout: Long)(predicate: (Seq[ClientResponse], Long) => Boolean)(implicit time: JTime): Boolean = { + pollUntilFound(timeout) { (responses, now) => + if (predicate(responses, now)) Some(true) + else None + }.fold(false)(_ => true) + } + + /** + * Invokes `client.poll` until `collect` returns `Some` or the timeout expires. + * + * It returns the result of `collect` if the call completes normally or `None` if the timeout expires. Exceptions + * thrown via `collect` are not handled and will bubble up. + * + * This method is useful for implementing blocking behaviour on top of the non-blocking `NetworkClient`, use it with + * care. + */ + private def pollUntilFound[T](timeout: Long)(collect: (Seq[ClientResponse], Long) => Option[T])(implicit time: JTime): Option[T] = { + + val methodStartTime = time.milliseconds() + val timeoutExpiryTime = methodStartTime + timeout + + @tailrec + def recurse(iterationStartTime: Long): Option[T] = { + val pollTimeout = if (timeout < 0) timeout else timeoutExpiryTime - iterationStartTime + val responses = client.poll(pollTimeout, iterationStartTime).asScala + val result = collect(responses, iterationStartTime) + if (result.isDefined) result + else { + val afterPollTime = time.milliseconds() + if (timeout < 0 || afterPollTime < timeoutExpiryTime) + recurse(afterPollTime) + else None + } + } + + recurse(methodStartTime) + } + +} diff --git a/core/src/main/scala/kafka/utils/ReplicationUtils.scala b/core/src/main/scala/kafka/utils/ReplicationUtils.scala index 715767380f7c2..d99629a30e279 100644 --- a/core/src/main/scala/kafka/utils/ReplicationUtils.scala +++ b/core/src/main/scala/kafka/utils/ReplicationUtils.scala @@ -18,22 +18,32 @@ package kafka.utils import kafka.api.LeaderAndIsr -import kafka.controller.LeaderIsrAndControllerEpoch -import org.apache.zookeeper.data.Stat +import kafka.common.TopicAndPartition +import kafka.controller.{IsrChangeNotificationListener, LeaderIsrAndControllerEpoch} import org.I0Itec.zkclient.ZkClient +import org.apache.zookeeper.data.Stat -import scala.Some import scala.collection._ object ReplicationUtils extends Logging { + private 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 - ZkUtils.conditionalUpdatePersistentPath(zkClient, path, newLeaderData, zkVersion, Some(checkLeaderAndIsrZkData)) + val updatePersistentPath: (Boolean, Int) = ZkUtils.conditionalUpdatePersistentPath(zkClient, path, newLeaderData, zkVersion, Some(checkLeaderAndIsrZkData)) + updatePersistentPath + } + + def propagateIsrChanges(zkClient: ZkClient, isrChangeSet: Set[TopicAndPartition]): Unit = { + val isrChangeNotificationPath: String = ZkUtils.createSequentialPersistentPath( + zkClient, ZkUtils.IsrChangeNotificationPath + "/" + IsrChangeNotificationPrefix, + generateIsrChangeJson(isrChangeSet)) + debug("Added " + isrChangeNotificationPath + " for " + isrChangeSet) } def checkLeaderAndIsrZkData(zkClient: ZkClient, path: String, expectedLeaderAndIsrInfo: String): (Boolean,Int) = { @@ -61,30 +71,27 @@ object ReplicationUtils extends Logging { def getLeaderIsrAndEpochForPartition(zkClient: ZkClient, topic: String, partition: Int):Option[LeaderIsrAndControllerEpoch] = { val leaderAndIsrPath = ZkUtils.getTopicPartitionLeaderAndIsrPath(topic, partition) - val leaderAndIsrInfo = ZkUtils.readDataMaybeNull(zkClient, leaderAndIsrPath) - val leaderAndIsrOpt = leaderAndIsrInfo._1 - val stat = leaderAndIsrInfo._2 - leaderAndIsrOpt match { - case Some(leaderAndIsrStr) => parseLeaderAndIsr(leaderAndIsrStr, leaderAndIsrPath, stat) - case None => None - } + 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) 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 leaderAndIsrPath %s".format(leader, epoch, - isr.toString(), zkPathVersion, path)) - Some(LeaderIsrAndControllerEpoch(LeaderAndIsr(leader, epoch, isr, zkPathVersion), controllerEpoch)) - case None => None - } + 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))} + } + + private def generateIsrChangeJson(isrChanges: Set[TopicAndPartition]): String = { + val partitions = isrChanges.map(tp => Map("topic" -> tp.topic, "partition" -> tp.partition)).toArray + Json.encode(Map("version" -> IsrChangeNotificationListener.version, "partitions" -> partitions)) } } diff --git a/core/src/main/scala/kafka/utils/ShutdownableThread.scala b/core/src/main/scala/kafka/utils/ShutdownableThread.scala index fc226c863095b..dc467975ff18d 100644 --- a/core/src/main/scala/kafka/utils/ShutdownableThread.scala +++ b/core/src/main/scala/kafka/utils/ShutdownableThread.scala @@ -51,6 +51,9 @@ abstract class ShutdownableThread(val name: String, val isInterruptible: Boolean info("Shutdown completed") } + /** + * This method is repeatedly invoked until the thread shuts down or this method throws an exception + */ def doWork(): Unit override def run(): Unit = { diff --git a/core/src/main/scala/kafka/utils/Throttler.scala b/core/src/main/scala/kafka/utils/Throttler.scala index d1a144d788291..998ade14ef82d 100644 --- a/core/src/main/scala/kafka/utils/Throttler.scala +++ b/core/src/main/scala/kafka/utils/Throttler.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package kafka.utils; +package kafka.utils import kafka.metrics.KafkaMetricsGroup import java.util.concurrent.TimeUnit @@ -95,4 +95,4 @@ object Throttler { } } } -} \ No newline at end of file +} diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala deleted file mode 100644 index 29d5a17d4a03c..0000000000000 --- a/core/src/main/scala/kafka/utils/Utils.scala +++ /dev/null @@ -1,569 +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.{ReadWriteLock, Lock} -import java.lang.management._ -import javax.management._ -import scala.collection._ -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 props = new Properties() - var propStream: InputStream = null - try { - propStream = new FileInputStream(filename) - props.load(propStream) - } finally { - if(propStream != null) - propStream.close - } - 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 - } - - /** - * 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) = if(n == Integer.MIN_VALUE) 0 else math.abs(n) - - /** - * 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 - * @throws 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 { - 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 - } -} 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 2f95d540c5329..34cab87034254 --- a/core/src/main/scala/kafka/utils/VerifiableProperties.scala +++ b/core/src/main/scala/kafka/utils/VerifiableProperties.scala @@ -35,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() } /** @@ -183,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)) diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index a7b1fdcb50d5c..e1cfa2e795964 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -17,12 +17,16 @@ package kafka.utils -import kafka.cluster.{Broker, Cluster} +import java.util.concurrent.CountDownLatch +import kafka.cluster._ import kafka.consumer.{ConsumerThreadId, TopicCount} -import org.I0Itec.zkclient.ZkClient -import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException, +import kafka.server.ConfigType +import org.I0Itec.zkclient.{ZkClient,ZkConnection} +import org.I0Itec.zkclient.exception.{ZkException, 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 org.apache.zookeeper.data.Stat @@ -30,22 +34,40 @@ import kafka.admin._ import kafka.common.{KafkaException, NoEpochForPartitionException} import kafka.controller.ReassignedPartitionsContext import kafka.controller.KafkaController -import scala.Some import kafka.controller.LeaderIsrAndControllerEpoch import kafka.common.TopicAndPartition -import scala.collection + +import org.apache.zookeeper.AsyncCallback.{DataCallback,StringCallback} +import org.apache.zookeeper.CreateMode +import org.apache.zookeeper.KeeperException +import org.apache.zookeeper.KeeperException.Code +import org.apache.zookeeper.ZooDefs.Ids +import org.apache.zookeeper.ZooKeeper + object ZkUtils extends Logging { val ConsumersPath = "/consumers" val BrokerIdsPath = "/brokers/ids" val BrokerTopicsPath = "/brokers/topics" - val TopicConfigPath = "/config/topics" - val TopicConfigChangesPath = "/config/changes" val ControllerPath = "/controller" val ControllerEpochPath = "/controller_epoch" val ReassignPartitionsPath = "/admin/reassign_partitions" val DeleteTopicsPath = "/admin/delete_topics" val PreferredReplicaLeaderElectionPath = "/admin/preferred_replica_election" + val BrokerSequenceIdPath = "/brokers/seqid" + val IsrChangeNotificationPath = "/isr_change_notification" + val EntityConfigPath = "/config" + val EntityConfigChangesPath = "/config/changes" + // These are persistent ZK paths that should exist on kafka broker startup. + val persistentZkPaths = Seq(ConsumersPath, + BrokerIdsPath, + BrokerTopicsPath, + EntityConfigChangesPath, + ZkUtils.getEntityConfigRootPath(ConfigType.Topic), + ZkUtils.getEntityConfigRootPath(ConfigType.Client), + DeleteTopicsPath, + BrokerSequenceIdPath, + IsrChangeNotificationPath) def getTopicPath(topic: String): String = { BrokerTopicsPath + "/" + topic @@ -55,8 +77,11 @@ object ZkUtils extends Logging { getTopicPath(topic) + "/partitions" } - def getTopicConfigPath(topic: String): String = - TopicConfigPath + "/" + topic + def getEntityConfigRootPath(entityType: String): String = + EntityConfigPath + "/" + entityType + + def getEntityConfigPath(entityType: String, entity: String): String = + getEntityConfigRootPath(entityType) + "/" + entity def getDeleteTopicPath(topic: String): String = DeleteTopicsPath + "/" + topic @@ -82,12 +107,16 @@ object ZkUtils extends Logging { brokerIds.map(_.toInt).map(getBrokerInfo(zkClient, _)).filter(_.isDefined).map(_.get) } + def getAllBrokerEndPointsForChannel(zkClient: ZkClient, protocolType: SecurityProtocol): Seq[BrokerEndPoint] = { + getAllBrokersInCluster(zkClient).map(_.getBrokerEndPoint(protocolType)) + } + def getLeaderAndIsrForPartition(zkClient: ZkClient, topic: String, partition: Int):Option[LeaderAndIsr] = { ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition).map(_.leaderAndIsr) } def setupCommonPaths(zkClient: ZkClient) { - for(path <- Seq(ConsumersPath, BrokerIdsPath, BrokerTopicsPath, TopicConfigChangesPath, TopicConfigPath, DeleteTopicsPath)) + for(path <- persistentZkPaths) makeSurePersistentPathExists(zkClient, path) } @@ -122,6 +151,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 */ @@ -158,31 +195,38 @@ 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, zkConnection: ZkConnection, id: Int, host: String, port: Int, advertisedEndpoints: immutable.Map[SecurityProtocol, EndPoint], 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) - try { - createEphemeralPathExpectConflictHandleZKBug(zkClient, brokerIdPath, brokerInfo, expectedBroker, - (brokerString: String, broker: Any) => Broker.createBroker(broker.asInstanceOf[Broker].id, brokerString).equals(broker.asInstanceOf[Broker]), - timeout) + val brokerInfo = Json.encode(Map("version" -> 2, "host" -> host, "port" -> port, "endpoints"->advertisedEndpoints.values.map(_.connectionString).toArray, "jmx_port" -> jmxPort, "timestamp" -> timestamp)) + registerBrokerInZk(zkClient, zkConnection, brokerIdPath, brokerInfo) + info("Registered broker %d at path %s with addresses: %s".format(id, brokerIdPath, advertisedEndpoints.mkString(","))) + } + + private def registerBrokerInZk(zkClient: ZkClient, zkConnection: ZkConnection, brokerIdPath: String, brokerInfo: String) { + try { + val zkCheckedEphemeral = new ZKCheckedEphemeral(brokerIdPath, + brokerInfo, + zkConnection.getZookeeper) + zkCheckedEphemeral.create() } 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 deregisterBrokerInZk(zkClient: ZkClient, id: Int) { - val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id - deletePath(zkClient, brokerIdPath) - info("Deregistered broker %d at path %s.".format(id, brokerIdPath)) } def getConsumerPartitionOwnerPath(group: String, topic: String, partition: Int): String = { @@ -208,7 +252,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 } /** @@ -216,8 +260,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) + } } /** @@ -225,11 +270,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) } } } @@ -263,63 +308,22 @@ object ZkUtils extends Logging { } } - /** - * Create an ephemeral node with the given path and data. - * Throw NodeExistsException if node already exists. - * Handles the following ZK session timeout bug: - * - * https://issues.apache.org/jira/browse/ZOOKEEPER-1740 - * - * Upon receiving a NodeExistsException, read the data from the conflicted path and - * trigger the checker function comparing the read data and the expected data, - * If the checker function returns true then the above bug might be encountered, back off and retry; - * otherwise re-throw the exception - */ - def createEphemeralPathExpectConflictHandleZKBug(zkClient: ZkClient, path: String, data: String, expectedCallerData: Any, checker: (String, Any) => Boolean, backoffTime: Int): Unit = { - while (true) { - try { - createEphemeralPathExpectConflict(zkClient, path, data) - return - } catch { - case e: ZkNodeExistsException => { - // An ephemeral node may still exist even after its corresponding session has expired - // due to a Zookeeper bug, in this case we need to retry writing until the previous node is deleted - // and hence the write succeeds without ZkNodeExistsException - ZkUtils.readDataMaybeNull(zkClient, path)._1 match { - case Some(writtenData) => { - if (checker(writtenData, expectedCallerData)) { - info("I wrote this conflicted ephemeral node [%s] at %s a while back in a different session, ".format(data, path) - + "hence I will backoff for this node to be deleted by Zookeeper and retry") - - Thread.sleep(backoffTime) - } else { - throw e - } - } - case None => // the node disappeared; retry creating the ephemeral node immediately - } - } - case e2: Throwable => throw e2 - } - } - } - /** * Create an persistent node with the given path and data. Create parents if necessary. */ def createPersistentPath(client: ZkClient, path: String, data: String = ""): Unit = { 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) } /** @@ -334,7 +338,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) @@ -405,7 +409,7 @@ 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 } @@ -436,7 +440,7 @@ object ZkUtils extends Logging { 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 { @@ -474,7 +478,7 @@ object ZkUtils extends Logging { try { client.getChildren(path) } catch { - case e: ZkNoNodeException => return Nil + case e: ZkNoNodeException => Nil case e2: Throwable => throw e2 } } @@ -575,23 +579,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] = { @@ -627,7 +636,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) @@ -691,6 +700,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) @@ -699,6 +732,17 @@ object ZkUtils extends Logging { topics } + /** + * Returns all the entities whose configs have been overridden. + */ + def getAllEntitiesWithConfig(zkClient: ZkClient, entityType: String): Seq[String] = { + val entities = ZkUtils.getChildrenParentMayNotExist(zkClient, getEntityConfigRootPath(entityType)) + if(entities == null) + Seq.empty[String] + else + entities + } + def getAllPartitions(zkClient: ZkClient): Set[TopicAndPartition] = { val topics = ZkUtils.getChildrenParentMayNotExist(zkClient, BrokerTopicsPath) if(topics == null) Set.empty[TopicAndPartition] @@ -708,9 +752,40 @@ object ZkUtils extends Logging { }.flatten.toSet } } + + 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 + } + } + } + + def createZkClient(zkUrl: String, sessionTimeout: Int, connectionTimeout: Int): ZkClient = { + val zkClient = new ZkClient(zkUrl, sessionTimeout, connectionTimeout, ZKStringSerializer) + zkClient + } + + def createZkClientAndConnection(zkUrl: String, sessionTimeout: Int, connectionTimeout: Int): (ZkClient, ZkConnection) = { + val zkConnection = new ZkConnection(zkUrl, sessionTimeout) + val zkClient = new ZkClient(zkConnection, connectionTimeout, ZKStringSerializer) + (zkClient, zkConnection) + } } -object ZKStringSerializer extends ZkSerializer { +private object ZKStringSerializer extends ZkSerializer { @throws(classOf[ZkMarshallingError]) def serialize(data : Object) : Array[Byte] = data.asInstanceOf[String].getBytes("UTF-8") @@ -728,11 +803,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 } @@ -749,3 +826,195 @@ 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) + } +} + +/** + * Creates an ephemeral znode checking the session owner + * in the case of conflict. In the regular case, the + * znode is created and the create call returns OK. If + * the call receives a node exists event, then it checks + * if the session matches. If it does, then it returns OK, + * and otherwise it fails the operation. + */ + +class ZKCheckedEphemeral(path: String, + data: String, + zkHandle: ZooKeeper) extends Logging { + private val createCallback = new CreateCallback + private val getDataCallback = new GetDataCallback + val latch: CountDownLatch = new CountDownLatch(1) + var result: Code = Code.OK + + private class CreateCallback extends StringCallback { + def processResult(rc: Int, + path: String, + ctx: Object, + name: String) { + Code.get(rc) match { + case Code.OK => + setResult(Code.OK) + case Code.CONNECTIONLOSS => + // try again + createEphemeral + case Code.NONODE => + error("No node for path %s (could be the parent missing)".format(path)) + setResult(Code.NONODE) + case Code.NODEEXISTS => + zkHandle.getData(path, false, getDataCallback, null) + case Code.SESSIONEXPIRED => + error("Session has expired while creating %s".format(path)) + setResult(Code.SESSIONEXPIRED) + case _ => + warn("ZooKeeper event while creating registration node: %s %s".format(path, Code.get(rc))) + setResult(Code.get(rc)) + } + } + } + + private class GetDataCallback extends DataCallback { + def processResult(rc: Int, + path: String, + ctx: Object, + readData: Array[Byte], + stat: Stat) { + Code.get(rc) match { + case Code.OK => + if (stat.getEphemeralOwner != zkHandle.getSessionId) + setResult(Code.NODEEXISTS) + else + setResult(Code.OK) + case Code.NONODE => + info("The ephemeral node [%s] at %s has gone away while reading it, ".format(data, path)) + createEphemeral + case Code.SESSIONEXPIRED => + error("Session has expired while reading znode %s".format(path)) + setResult(Code.SESSIONEXPIRED) + case _ => + warn("ZooKeeper event while getting znode data: %s %s".format(path, Code.get(rc))) + setResult(Code.get(rc)) + } + } + } + + private def createEphemeral() { + zkHandle.create(path, + ZKStringSerializer.serialize(data), + Ids.OPEN_ACL_UNSAFE, + CreateMode.EPHEMERAL, + createCallback, + null) + } + + private def createRecursive(prefix: String, suffix: String) { + debug("Path: %s, Prefix: %s, Suffix: %s".format(path, prefix, suffix)) + if(suffix.isEmpty()) { + createEphemeral + } else { + zkHandle.create(prefix, + new Array[Byte](0), + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT, + new StringCallback() { + def processResult(rc : Int, + path : String, + ctx : Object, + name : String) { + Code.get(rc) match { + case Code.OK | Code.NODEEXISTS => + // Nothing to do + case Code.CONNECTIONLOSS => + // try again + val suffix = ctx.asInstanceOf[String] + createRecursive(path, suffix) + case Code.NONODE => + error("No node for path %s (could be the parent missing)".format(path)) + setResult(Code.get(rc)) + case Code.SESSIONEXPIRED => + error("Session has expired while creating %s".format(path)) + setResult(Code.get(rc)) + case _ => + warn("ZooKeeper event while creating registration node: %s %s".format(path, Code.get(rc))) + setResult(Code.get(rc)) + } + } + }, + suffix) + // Update prefix and suffix + val index = suffix.indexOf('/', 1) match { + case -1 => suffix.length + case x : Int => x + } + // Get new prefix + val newPrefix = prefix + suffix.substring(0, index) + // Get new suffix + val newSuffix = suffix.substring(index, suffix.length) + createRecursive(newPrefix, newSuffix) + } + } + + private def setResult(code: Code) { + result = code + latch.countDown() + } + + private def waitUntilResolved(): Code = { + latch.await() + result + } + + def create() { + val index = path.indexOf('/', 1) match { + case -1 => path.length + case x : Int => x + } + val prefix = path.substring(0, index) + val suffix = path.substring(index, path.length) + debug("Path: %s, Prefix: %s, Suffix: %s".format(path, prefix, suffix)) + createRecursive(prefix, suffix) + val result = waitUntilResolved() + info("Result of znode creation is: %s".format(result)) + result match { + case Code.OK => + // Nothing to do + case _ => + throw ZkException.create(KeeperException.create(result)) + } + } +} 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/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala new file mode 100644 index 0000000000000..16d7c26f800c6 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -0,0 +1,179 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package kafka.api + +import java.util + +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.errors.{IllegalGenerationException, UnknownConsumerIdException} +import org.apache.kafka.common.TopicPartition +import org.junit.Assert._ +import org.junit.{Test, Before} + +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, "100") + this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "30") + this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + + override def generateConfigs() = { + FixedPortTestUtils.createBrokerConfigs(serverCount, zkConnect,enableControlledShutdown = false) + .map(KafkaConfig.fromProps(_, serverConfig)) + } + + @Before + override def setUp() { + super.setUp() + + // create the test topic with all the brokers as replicas + TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) + } + + @Test + def testConsumptionWithBrokerFailures() = consumeWithBrokerFailures(10) + + /* + * 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 = 0L + val consumer = this.consumers(0) + + consumer.subscribe(List(topic), new ConsumerRebalanceListener { + override def onPartitionsAssigned(partitions: util.Collection[TopicPartition]) { + // TODO: until KAFKA-2017 is merged, we have to handle the case in which the + // the commit fails on prior to rebalancing on coordinator fail-over. + consumer.seek(tp, consumed) + } + override def onPartitionsRevoked(partitions: util.Collection[TopicPartition]) {} + }) + + val scheduler = new BounceBrokerScheduler(numIters) + scheduler.start() + + while (scheduler.isRunning.get()) { + for (record <- consumer.poll(100)) { + assertEquals(consumed, record.offset()) + consumed += 1 + } + + try { + consumer.commitSync() + assertEquals(consumer.position(tp), consumer.committed(tp).offset) + + if (consumer.position(tp) == numRecords) { + consumer.seekToBeginning() + consumed = 0 + } + } catch { + // TODO: should be no need to catch these exceptions once KAFKA-2017 is + // merged since coordinator fail-over will not cause a rebalance + case _: UnknownConsumerIdException | _: IllegalGenerationException => + } + } + scheduler.shutdown() + } + + @Test + def testSeekAndCommitWithBrokerFailures() = seekAndCommitWithBrokerFailures(5) + + def seekAndCommitWithBrokerFailures(numIters: Int) { + val numRecords = 1000 + sendRecords(numRecords) + this.producers.foreach(_.close) + + val consumer = this.consumers(0) + consumer.assign(List(tp)) + consumer.seek(tp, 0) + + // wait until all the followers have synced the last HW with leader + TestUtils.waitUntilTrue(() => servers.forall(server => + server.replicaManager.getReplica(tp.topic(), tp.partition()).get.highWatermark.messageOffset == numRecords + ), "Failed to update high watermark for followers after timeout") + + 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.commitSync() + assertEquals(consumer.position(tp), consumer.committed(tp).offset) + } + } + } + + 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..166b914a58496 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala @@ -0,0 +1,565 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES 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.regex.Pattern +import java.{lang, util} + +import org.apache.kafka.clients.consumer._ +import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} +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 org.junit.{Test, Before} + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +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") + this.consumerConfig.setProperty(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "100") + + @Before + override def setUp() { + super.setUp() + + // create the test topic with all the brokers as replicas + TestUtils.createTopic(this.zkClient, topic, 2, serverCount, this.servers) + } + + @Test + def testSimpleConsumption() { + val numRecords = 10000 + sendRecords(numRecords) + + assertEquals(0, this.consumers(0).assignment.size) + this.consumers(0).assign(List(tp)) + assertEquals(1, this.consumers(0).assignment.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).commitAsync(commitCallback) + + // shouldn't make progress until poll is invoked + Thread.sleep(10) + assertEquals(0, commitCallback.count) + awaitCommitCallback(this.consumers(0), commitCallback) + } + + @Test + def testAutoCommitOnClose() { + this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true"); + val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer()) + + val numRecords = 10000 + sendRecords(numRecords) + + consumer0.subscribe(List(topic)) + + val assignment = Set(tp, tp2) + TestUtils.waitUntilTrue(() => { + consumer0.poll(50) + consumer0.assignment() == assignment.asJava + }, s"Expected partitions ${assignment.asJava} but actually got ${consumer0.assignment()}") + + // should auto-commit seeked positions before closing + consumer0.seek(tp, 300) + consumer0.seek(tp2, 500) + consumer0.close() + + // now we should see the committed positions from another consumer + assertEquals(300, this.consumers(0).committed(tp).offset) + assertEquals(500, this.consumers(0).committed(tp2).offset) + } + + @Test + def testAutoCommitOnRebalance() { + val topic2 = "topic2" + TestUtils.createTopic(this.zkClient, topic2, 2, serverCount, this.servers) + + this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true"); + val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer()) + + val numRecords = 10000 + sendRecords(numRecords) + + consumer0.subscribe(List(topic)) + + val assignment = Set(tp, tp2) + TestUtils.waitUntilTrue(() => { + consumer0.poll(50) + consumer0.assignment() == assignment.asJava + }, s"Expected partitions ${assignment.asJava} but actually got ${consumer0.assignment()}") + + consumer0.seek(tp, 300) + consumer0.seek(tp2, 500) + + // change subscription to trigger rebalance + consumer0.subscribe(List(topic, topic2)) + + val newAssignment = Set(tp, tp2, new TopicPartition(topic2, 0), new TopicPartition(topic2, 1)) + TestUtils.waitUntilTrue(() => { + consumer0.poll(50) + consumer0.assignment() == newAssignment.asJava + }, s"Expected partitions ${newAssignment.asJava} but actually got ${consumer0.assignment()}") + + // after rebalancing, we should have reset to the committed positions + assertEquals(300, consumer0.committed(tp).offset) + assertEquals(500, consumer0.committed(tp2).offset) + } + + @Test + def testPatternSubscription() { + val numRecords = 10000 + sendRecords(numRecords) + + val topic1: String = "tblablac" // matches subscribed pattern + TestUtils.createTopic(this.zkClient, topic1, 2, serverCount, this.servers) + sendRecords(1000, new TopicPartition(topic1, 0)) + sendRecords(1000, new TopicPartition(topic1, 1)) + + val topic2: String = "tblablak" // does not match subscribed pattern + TestUtils.createTopic(this.zkClient, topic2, 2, serverCount, this.servers) + sendRecords(1000, new TopicPartition(topic2, 0)) + sendRecords(1000, new TopicPartition(topic2, 1)) + + val topic3: String = "tblab1" // does not match subscribed pattern + TestUtils.createTopic(this.zkClient, topic3, 2, serverCount, this.servers) + sendRecords(1000, new TopicPartition(topic3, 0)) + sendRecords(1000, new TopicPartition(topic3, 1)) + + assertEquals(0, this.consumers(0).assignment().size) + + val pattern: Pattern = Pattern.compile("t.*c") + this.consumers(0).subscribe(pattern, new TestConsumerReassignmentListener) + this.consumers(0).poll(50) + + var subscriptions = Set( + new TopicPartition(topic, 0), + new TopicPartition(topic, 1), + new TopicPartition(topic1, 0), + new TopicPartition(topic1, 1)) + + TestUtils.waitUntilTrue(() => { + this.consumers(0).poll(50) + this.consumers(0).assignment() == subscriptions.asJava + }, s"Expected partitions ${subscriptions.asJava} but actually got ${this.consumers(0).assignment()}") + + val topic4: String = "tsomec" // matches subscribed pattern + TestUtils.createTopic(this.zkClient, topic4, 2, serverCount, this.servers) + sendRecords(1000, new TopicPartition(topic4, 0)) + sendRecords(1000, new TopicPartition(topic4, 1)) + + subscriptions = subscriptions ++ Set( + new TopicPartition(topic4, 0), + new TopicPartition(topic4, 1)) + + + TestUtils.waitUntilTrue(() => { + this.consumers(0).poll(50) + this.consumers(0).assignment() == subscriptions.asJava + }, s"Expected partitions ${subscriptions.asJava} but actually got ${this.consumers(0).assignment()}") + + this.consumers(0).unsubscribe() + assertEquals(0, this.consumers(0).assignment().size) + } + + @Test + def testPatternUnsubscription() { + val numRecords = 10000 + sendRecords(numRecords) + + val topic1: String = "tblablac" // matches subscribed pattern + TestUtils.createTopic(this.zkClient, topic1, 2, serverCount, this.servers) + sendRecords(1000, new TopicPartition(topic1, 0)) + sendRecords(1000, new TopicPartition(topic1, 1)) + + assertEquals(0, this.consumers(0).assignment().size) + + this.consumers(0).subscribe(Pattern.compile("t.*c"), new TestConsumerReassignmentListener) + this.consumers(0).poll(50) + + val subscriptions = Set( + new TopicPartition(topic, 0), + new TopicPartition(topic, 1), + new TopicPartition(topic1, 0), + new TopicPartition(topic1, 1)) + + TestUtils.waitUntilTrue(() => { + this.consumers(0).poll(50) + this.consumers(0).assignment() == subscriptions.asJava + }, s"Expected partitions ${subscriptions.asJava} but actually got ${this.consumers(0).assignment()}") + + this.consumers(0).unsubscribe() + assertEquals(0, this.consumers(0).assignment().size) + } + + @Test + def testCommitSpecifiedOffsets() { + sendRecords(5, tp) + sendRecords(7, tp2) + + this.consumers(0).assign(List(tp, 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).commitSync(Map[TopicPartition,OffsetAndMetadata]((tp, new OffsetAndMetadata(3L))).asJava) + assertEquals(3, this.consumers(0).committed(tp).offset) + assertNull(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).commitSync(Map[TopicPartition,OffsetAndMetadata]((tp2, new OffsetAndMetadata(5L))).asJava) + assertEquals(3, this.consumers(0).committed(tp).offset) + assertEquals(5, this.consumers(0).committed(tp2).offset) + + // Using async should pick up the committed changes after commit completes + val commitCallback = new CountConsumerCommitCallback() + this.consumers(0).commitAsync(Map[TopicPartition,OffsetAndMetadata]((tp2, new OffsetAndMetadata(7L))).asJava, commitCallback) + awaitCommitCallback(this.consumers(0), commitCallback) + assertEquals(7, this.consumers(0).committed(tp2).offset) + } + + @Test + def testAutoOffsetReset() { + sendRecords(1) + this.consumers(0).assign(List(tp)) + consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0) + } + + @Test + def testSeek() { + val consumer = this.consumers(0) + val totalRecords = 50L + sendRecords(totalRecords.toInt) + consumer.assign(List(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) + } + + @Test + def testGroupConsumption() { + sendRecords(10) + this.consumers(0).subscribe(List(topic)) + consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0) + } + + + @Test + def testCommitMetadata() { + this.consumers(0).assign(List(tp)) + + // sync commit + val syncMetadata = new OffsetAndMetadata(5, "foo") + this.consumers(0).commitSync(Map((tp, syncMetadata))) + assertEquals(syncMetadata, this.consumers(0).committed(tp)) + + // async commit + val asyncMetadata = new OffsetAndMetadata(10, "bar") + val callback = new CountConsumerCommitCallback + this.consumers(0).commitAsync(Map((tp, asyncMetadata)), callback) + awaitCommitCallback(this.consumers(0), callback) + + assertEquals(asyncMetadata, this.consumers(0).committed(tp)) + } + + 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).assign(List(tp)) + + assertEquals("position() on a partition that we are subscribed to should reset the offset", 0L, this.consumers(0).position(tp)) + this.consumers(0).commitSync() + assertEquals(0L, this.consumers(0).committed(tp).offset) + + consumeRecords(this.consumers(0), 5, 0) + assertEquals("After consuming 5 records, position should be 5", 5L, this.consumers(0).position(tp)) + this.consumers(0).commitSync() + assertEquals("Committed offset should be returned", 5L, this.consumers(0).committed(tp).offset) + + sendRecords(1) + + // another consumer in the same group should get the same position + this.consumers(1).assign(List(tp)) + consumeRecords(this.consumers(1), 1, 5) + } + + @Test + 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.size) + assertNull(this.consumers(0).partitionsFor("non-exist-topic")) + } + + @Test + def testListTopics() { + val numParts = 2 + val topic1: String = "part-test-topic-1" + val topic2: String = "part-test-topic-2" + val topic3: String = "part-test-topic-3" + TestUtils.createTopic(this.zkClient, topic1, numParts, 1, this.servers) + TestUtils.createTopic(this.zkClient, topic2, numParts, 1, this.servers) + TestUtils.createTopic(this.zkClient, topic3, numParts, 1, this.servers) + + val topics = this.consumers.head.listTopics() + assertNotNull(topics) + assertEquals(5, topics.size()) + assertEquals(5, topics.keySet().size()) + assertEquals(2, topics.get(topic1).size) + assertEquals(2, topics.get(topic2).size) + assertEquals(2, topics.get(topic3).size) + } + + @Test + def testPartitionReassignmentCallback() { + val listener = new TestConsumerReassignmentListener() + this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100"); // timeout quickly to avoid slow test + this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "30"); + val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer()) + consumer0.subscribe(List(topic), listener) + + // the initial subscription should cause a callback execution + while(listener.callsToAssigned == 0) + consumer0.poll(50) + + // get metadata for the topic + var parts = consumer0.partitionsFor(ConsumerCoordinator.OffsetsTopicName).asScala + while(parts == null) + parts = consumer0.partitionsFor(ConsumerCoordinator.OffsetsTopicName).asScala + 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(listener.callsToAssigned < 2) + consumer0.poll(50) + + assertEquals(2, listener.callsToAssigned) + assertEquals(2, listener.callsToRevoked) + + consumer0.close() + } + + @Test + def testUnsubscribeTopic() { + + this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100"); // timeout quickly to avoid slow test + this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "30"); + val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer()) + + try { + val listener = new TestConsumerReassignmentListener() + consumer0.subscribe(List(topic), listener) + + // the initial subscription should cause a callback execution + while (listener.callsToAssigned == 0) + consumer0.poll(50) + + consumer0.subscribe(List()) + assertEquals(0, consumer0.assignment.size()) + } finally { + consumer0.close() + } + } + + @Test + def testExpandingTopicSubscriptions() { + val otherTopic = "other" + val subscriptions = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1)) + val expandedSubscriptions = subscriptions ++ Set(new TopicPartition(otherTopic, 0), new TopicPartition(otherTopic, 1)) + this.consumers(0).subscribe(List(topic)) + TestUtils.waitUntilTrue(() => { + this.consumers(0).poll(50) + this.consumers(0).assignment == subscriptions.asJava + }, s"Expected partitions ${subscriptions.asJava} but actually got ${this.consumers(0).assignment}") + + TestUtils.createTopic(this.zkClient, otherTopic, 2, serverCount, this.servers) + this.consumers(0).subscribe(List(topic, otherTopic)) + TestUtils.waitUntilTrue(() => { + this.consumers(0).poll(50) + this.consumers(0).assignment == expandedSubscriptions.asJava + }, s"Expected partitions ${expandedSubscriptions.asJava} but actually got ${this.consumers(0).assignment}") + } + + @Test + def testShrinkingTopicSubscriptions() { + val otherTopic = "other" + TestUtils.createTopic(this.zkClient, otherTopic, 2, serverCount, this.servers) + val subscriptions = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1), new TopicPartition(otherTopic, 0), new TopicPartition(otherTopic, 1)) + val shrunkenSubscriptions = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1)) + this.consumers(0).subscribe(List(topic, otherTopic)) + TestUtils.waitUntilTrue(() => { + this.consumers(0).poll(50) + this.consumers(0).assignment == subscriptions.asJava + }, s"Expected partitions ${subscriptions.asJava} but actually got ${this.consumers(0).assignment}") + + this.consumers(0).subscribe(List(topic)) + TestUtils.waitUntilTrue(() => { + this.consumers(0).poll(50) + this.consumers(0).assignment == shrunkenSubscriptions.asJava + }, s"Expected partitions ${shrunkenSubscriptions.asJava} but actually got ${this.consumers(0).assignment}") + } + + @Test + def testPartitionPauseAndResume() { + sendRecords(5) + this.consumers(0).assign(List(tp)) + consumeRecords(this.consumers(0), 5, 0) + this.consumers(0).pause(tp) + sendRecords(5) + assertTrue(this.consumers(0).poll(0).isEmpty) + this.consumers(0).resume(tp) + consumeRecords(this.consumers(0), 5, 5) + } + + @Test + def testPauseStateNotPreservedByRebalance() { + this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100"); // timeout quickly to avoid slow test + this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "30"); + val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer()) + + sendRecords(5) + consumer0.subscribe(List(topic)) + consumeRecords(consumer0, 5, 0) + consumer0.pause(tp) + + // subscribe to a new topic to trigger a rebalance + consumer0.subscribe(List("topic2")) + + // after rebalance, our position should be reset and our pause state lost, + // so we should be able to consume from the beginning + consumeRecords(consumer0, 0, 5) + } + + private class TestConsumerReassignmentListener extends ConsumerRebalanceListener { + var callsToAssigned = 0 + var callsToRevoked = 0 + def onPartitionsAssigned(partitions: java.util.Collection[TopicPartition]) { + info("onPartitionsAssigned called.") + callsToAssigned += 1 + } + def onPartitionsRevoked(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).asScala) + 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(50) + assertEquals(startCount + 1, commitCallback.count) + } + + private class CountConsumerCommitCallback extends OffsetCommitCallback { + var count = 0 + + override def onComplete(offsets: util.Map[TopicPartition, OffsetAndMetadata], 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..8080b085a38db --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.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.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 org.junit.{After, Before} +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) + } + + @Before + 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) + } + + @After + 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..2dbb9dcd94b66 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/ProducerBounceTest.scala @@ -0,0 +1,163 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES 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.{After, Before, 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" + + @Before + 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) + } + + @After + 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 old mode 100644 new mode 100755 index 17e2c6e9dfd78..87db2551c6c06 --- a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala @@ -19,7 +19,6 @@ 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 @@ -32,33 +31,31 @@ import kafka.server.{KafkaConfig, KafkaServer} import kafka.consumer.SimpleConsumer import kafka.message.Message import kafka.zk.ZooKeeperTestHarness -import kafka.utils.{Utils, TestUtils} - -import scala.Array +import kafka.utils.{CoreUtils, TestUtils} @RunWith(value = classOf[Parameterized]) -class ProducerCompressionTest(compression: String) extends JUnit3Suite with ZooKeeperTestHarness { +class ProducerCompressionTest(compression: String) extends ZooKeeperTestHarness { private val brokerId = 0 - private val port = TestUtils.choosePort private var server: KafkaServer = null - private val props = TestUtils.createBrokerConfig(brokerId, port) - private val config = new KafkaConfig(props) - private val topic = "topic" - private val numRecords = 100 + 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 - Utils.rm(server.config.logDirs) + CoreUtils.rm(server.config.logDirs) super.tearDown() } @@ -71,10 +68,14 @@ class ProducerCompressionTest(compression: String) extends JUnit3Suite with ZooK def testCompression() { val props = new Properties() - props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getBrokerListStrFromConfigs(Seq(config))) + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getBrokerListStrFromServers(Seq(server))) props.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, compression) - var producer = new KafkaProducer(props) - val consumer = new SimpleConsumer("localhost", port, 100, 1024*1024, "") + 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 @@ -87,7 +88,7 @@ class ProducerCompressionTest(compression: String) extends JUnit3Suite with ZooK // make sure the returned messages are correct val responses = for (message <- messages) - yield producer.send(new ProducerRecord(topic, null, null, message)) + 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) @@ -121,8 +122,10 @@ object ProducerCompressionTest { @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 index 39f777baebb6c..e90818af916dd 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -17,65 +17,59 @@ package kafka.api -import kafka.common.Topic -import org.apache.kafka.common.errors.InvalidTopicException -import org.scalatest.junit.JUnit3Suite -import org.junit.Test -import org.junit.Assert._ +import java.util.concurrent.{ExecutionException, TimeUnit, TimeoutException} +import java.util.{Properties, Random} -import java.util.Random -import java.lang.Integer -import java.util.concurrent.{TimeoutException, TimeUnit, ExecutionException} - -import kafka.server.KafkaConfig -import kafka.utils.{TestZKUtils, ShutdownableThread, TestUtils} -import kafka.integration.KafkaServerTestHarness +import kafka.common.Topic import kafka.consumer.SimpleConsumer - -import org.apache.kafka.common.KafkaException +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.apache.kafka.common.KafkaException +import org.apache.kafka.common.errors.{InvalidTopicException, NotEnoughReplicasAfterAppendException, NotEnoughReplicasException} +import org.junit.Assert._ +import org.junit.{After, Before, Test} -class ProducerFailureHandlingTest extends JUnit3Suite with KafkaServerTestHarness { +class ProducerFailureHandlingTest extends KafkaServerTestHarness { private val producerBufferSize = 30000 private val serverMessageMaxBytes = producerBufferSize/2 val numServers = 2 - val configs = - for(props <- TestUtils.createBrokerConfigs(numServers, false)) - yield new KafkaConfig(props) { - override val zkConnect = TestZKUtils.zookeeperConnect - override val autoCreateTopicsEnable = false - override val messageMaxBytes = serverMessageMaxBytes - } + 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 = null - private var producer2: KafkaProducer = null - private var producer3: KafkaProducer = null - private var producer4: KafkaProducer = 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" + @Before override def setUp() { super.setUp() - // TODO: we need to migrate to new consumers when 0.9 is final - consumer1 = new SimpleConsumer("localhost", configs(0).port, 100, 1024*1024, "") - consumer2 = new SimpleConsumer("localhost", configs(1).port, 100, 1024*1024, "") - 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) } + @After override def tearDown() { - consumer1.close - consumer2.close - if (producer1 != null) producer1.close if (producer2 != null) producer2.close if (producer3 != null) producer3.close @@ -90,10 +84,10 @@ class ProducerFailureHandlingTest extends JUnit3Suite with KafkaServerTestHarnes @Test def testTooLargeRecordWithAckZero() { // create topic - TestUtils.createTopic(zkClient, topic1, 1, 2, servers) + TestUtils.createTopic(zkClient, topic1, 1, numServers, servers) // send a too-large record - val record = new ProducerRecord(topic1, null, "key".getBytes, new Array[Byte](serverMessageMaxBytes + 1)) + 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) } @@ -103,10 +97,10 @@ class ProducerFailureHandlingTest extends JUnit3Suite with KafkaServerTestHarnes @Test def testTooLargeRecordWithAckOne() { // create topic - TestUtils.createTopic(zkClient, topic1, 1, 2, servers) + TestUtils.createTopic(zkClient, topic1, 1, numServers, servers) // send a too-large record - val record = new ProducerRecord(topic1, null, "key".getBytes, new Array[Byte](serverMessageMaxBytes + 1)) + val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, new Array[Byte](serverMessageMaxBytes + 1)) intercept[ExecutionException] { producer2.send(record).get } @@ -118,7 +112,7 @@ class ProducerFailureHandlingTest extends JUnit3Suite with KafkaServerTestHarnes @Test def testNonExistentTopic() { // send a record with non-exist topic - val record = new ProducerRecord(topic2, null, "key".getBytes, "value".getBytes) + val record = new ProducerRecord[Array[Byte],Array[Byte]](topic2, null, "key".getBytes, "value".getBytes) intercept[ExecutionException] { producer1.send(record).get } @@ -137,71 +131,28 @@ class ProducerFailureHandlingTest extends JUnit3Suite with KafkaServerTestHarnes @Test def testWrongBrokerList() { // create topic - TestUtils.createTopic(zkClient, topic1, 1, 2, servers) + 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(topic1, null, "key".getBytes, "value".getBytes) + 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, 2, servers) - - // first send a message to make sure the metadata is refreshed - val record1 = new ProducerRecord(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(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, 2, servers) + TestUtils.createTopic(zkClient, topic1, 1, numServers, servers) // create a record with incorrect partition id, send should fail - val record = new ProducerRecord(topic1, new Integer(1), "key".getBytes, "value".getBytes) + val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, new Integer(1), "key".getBytes, "value".getBytes) intercept[IllegalArgumentException] { producer1.send(record) } @@ -219,9 +170,9 @@ class ProducerFailureHandlingTest extends JUnit3Suite with KafkaServerTestHarnes @Test def testSendAfterClosed() { // create topic - TestUtils.createTopic(zkClient, topic1, 1, 2, servers) + TestUtils.createTopic(zkClient, topic1, 1, numServers, servers) - val record = new ProducerRecord(topic1, null, "key".getBytes, "value".getBytes) + 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 @@ -244,62 +195,64 @@ class ProducerFailureHandlingTest extends JUnit3Suite with KafkaServerTestHarnes // re-close producer is fine } - /** - * With replication, producer should able able to find new leader after it detects broker failure - */ @Test - def testBrokerFailure() { - // create topic - val leaders = TestUtils.createTopic(zkClient, topic1, 1, 2, servers) - val partition = 0 - assertTrue("Leader of partition 0 of the topic should exist", leaders(partition).isDefined) - - val scheduler = new ProducerScheduler() - scheduler.start - - // rolling bounce brokers - for (i <- 0 until 2) { - 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) + 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]) + } - // Make sure the leader still exists after bouncing brokers - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic1, partition) + @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") + } } + } - scheduler.shutdown + @Test + def testNotEnoughReplicasAfterBrokerShutdown() { + val topicName = "minisrtest2" + val topicProps = new Properties() + topicProps.put("min.insync.replicas",numServers.toString) - // Make sure the producer do not see any exception - // when draining the left messages on shutdown - assertTrue(scheduler.failed == false) + TestUtils.createTopic(zkClient, topicName, 1, numServers, servers,topicProps) - // double check that the leader info has been propagated after consecutive bounces - val leader = TestUtils.waitUntilMetadataIsPropagated(servers, topic1, partition) + 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 - val fetchResponse = if(leader == configs(0).brokerId) { - consumer1.fetch(new FetchRequestBuilder().addFetch(topic1, partition, 0, Int.MaxValue).build()).messageSet(topic1, partition) - } else { - consumer2.fetch(new FetchRequestBuilder().addFetch(topic1, partition, 0, Int.MaxValue).build()).messageSet(topic1, partition) + // 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] && + !e.getCause.isInstanceOf[TimeoutException]) { + fail("Expected NotEnoughReplicasException or NotEnoughReplicasAfterAppendException when producing to topic " + + "with fewer brokers than min.insync.replicas, but saw " + e.getCause) + } } - val messages = fetchResponse.iterator.toList.map(_.message) - val uniqueMessages = messages.toSet - val uniqueMessageSize = uniqueMessages.size - - assertEquals("Should have fetched " + scheduler.sent + " unique messages", scheduler.sent, uniqueMessageSize) - } - - @Test(expected = classOf[InvalidTopicException]) - def testCannotSendToInternalTopic() { - producer1.send(new ProducerRecord(Topic.InternalTopics.head, "test".getBytes, "test".getBytes)).get + // restart the server + servers.head.startup() } private class ProducerScheduler extends ShutdownableThread("daemon-producer", false) @@ -313,7 +266,8 @@ class ProducerFailureHandlingTest extends JUnit3Suite with KafkaServerTestHarnes override def doWork(): Unit = { val responses = for (i <- sent+1 to sent+numRecords) - yield producer.send(new ProducerRecord(topic1, null, null, i.toString.getBytes)) + 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 { diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala index d407af9144ef6..637d6f3157a2c 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala @@ -15,31 +15,31 @@ * limitations under the License. */ -package kafka.api.test +package kafka.api -import java.lang.{Integer, IllegalArgumentException} +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.scalatest.junit.JUnit3Suite -import org.junit.Test +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.{After, Before, Test} -import kafka.server.KafkaConfig -import kafka.utils.{TestZKUtils, TestUtils} -import kafka.consumer.SimpleConsumer -import kafka.api.FetchRequestBuilder -import kafka.message.Message -import kafka.integration.KafkaServerTestHarness +class ProducerSendTest extends KafkaServerTestHarness { + val numServers = 2 + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.NumPartitionsProp, 4.toString) -class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { - val numServers = 2 - val configs = - for(props <- TestUtils.createBrokerConfigs(numServers, false)) - yield new KafkaConfig(props) { - override val zkConnect = TestZKUtils.zookeeperConnect - override val numPartitions = 4 - } + def generateConfigs() = + TestUtils.createBrokerConfigs(numServers, zkConnect, false).map(KafkaConfig.fromProps(_, overridingProps)) private var consumer1: SimpleConsumer = null private var consumer2: SimpleConsumer = null @@ -47,14 +47,16 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { private val topic = "topic" private val numRecords = 100 + @Before override def setUp() { super.setUp() // TODO: we need to migrate to new consumers when 0.9 is final - consumer1 = new SimpleConsumer("localhost", configs(0).port, 100, 1024*1024, "") - consumer2 = new SimpleConsumer("localhost", configs(1).port, 100, 1024*1024, "") + consumer1 = new SimpleConsumer("localhost", servers(0).boundPort(), 100, 1024*1024, "") + consumer2 = new SimpleConsumer("localhost", servers(1).boundPort(), 100, 1024*1024, "") } + @After override def tearDown() { consumer1.close() consumer2.close() @@ -62,13 +64,6 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { super.tearDown() } - class CheckErrorCallback extends Callback { - def onCompletion(metadata: RecordMetadata, exception: Exception) { - if (exception != null) - fail("Send callback returns the following exception", exception) - } - } - /** * testSendOffset checks the basic send API behavior * @@ -78,32 +73,45 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testSendOffset() { var producer = TestUtils.createNewProducer(brokerList) - - val callback = new CheckErrorCallback + 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(topic, new Integer(0), "key".getBytes, "value".getBytes) + 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(topic, new Integer(0), "key".getBytes, null) + 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(topic, new Integer(0), null, "value".getBytes) + 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(topic, null, "key".getBytes, "value".getBytes) + 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(null, new Integer(0), "key".getBytes, "value".getBytes) + 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 { @@ -113,7 +121,7 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { // non-blocking send a list of records for (i <- 1 to numRecords) - producer.send(record0) + 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) @@ -126,6 +134,55 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { } } + @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 * @@ -140,7 +197,7 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { TestUtils.createTopic(zkClient, topic, 1, 2, servers) // non-blocking send a list of records - val record0 = new ProducerRecord(topic, null, "key".getBytes, "value".getBytes) + 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) @@ -182,7 +239,7 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { val responses = for (i <- 1 to numRecords) - yield producer.send(new ProducerRecord(topic, partition, null, ("value" + i).getBytes)) + 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) @@ -228,7 +285,7 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { try { // Send a message to auto-create the topic - val record = new ProducerRecord(topic, null, "key".getBytes, "value".getBytes) + 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 @@ -241,4 +298,119 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { } } } + + /** + * 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, 1, 2, servers) + val leader = leaders(0) + + // create record + val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, 0, 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(record)) + // 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 + val responses = ((0 until numRecords) map (i => producer.send(record, 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 fetchResponse = if (leader.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 expectedNumRecords = (i + 1) * numRecords + assertEquals("Fetch response to partition 0 should have %d messages.".format(expectedNumRecords), + expectedNumRecords, fetchResponse.messageSet(topic, 0).size) + } + } finally { + if (producer != null) + producer.close() + } + } } diff --git a/core/src/test/scala/integration/kafka/api/QuotasTest.scala b/core/src/test/scala/integration/kafka/api/QuotasTest.scala new file mode 100644 index 0000000000000..38b3dbd102d9f --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/QuotasTest.scala @@ -0,0 +1,194 @@ +/** + * 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. + **/ + +package kafka.api + +import java.util.Properties + +import junit.framework.Assert +import kafka.consumer.SimpleConsumer +import kafka.integration.KafkaServerTestHarness +import kafka.server.{KafkaServer, KafkaConfig} +import kafka.utils.TestUtils +import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} +import org.apache.kafka.clients.producer._ +import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback +import org.apache.kafka.common.MetricName +import org.apache.kafka.common.metrics.KafkaMetric +import org.junit.Assert._ +import org.junit.{After, Before, Test} +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ + +import scala.collection.mutable + +class QuotasTest extends KafkaServerTestHarness { + private val producerBufferSize = 300000 + private val producerId1 = "QuotasTestProducer-1" + private val producerId2 = "QuotasTestProducer-2" + private val consumerId1 = "QuotasTestConsumer-1" + private val consumerId2 = "QuotasTestConsumer-2" + + val numServers = 2 + val overridingProps = new Properties() + + // Low enough quota that a producer sending a small payload in a tight loop should get throttled + overridingProps.put(KafkaConfig.ProducerQuotaBytesPerSecondDefaultProp, "8000") + overridingProps.put(KafkaConfig.ConsumerQuotaBytesPerSecondDefaultProp, "2500") + + // un-throttled + overridingProps.put(KafkaConfig.ProducerQuotaBytesPerSecondOverridesProp, producerId2 + "=" + Long.MaxValue) + overridingProps.put(KafkaConfig.ConsumerQuotaBytesPerSecondOverridesProp, consumerId2 + "=" + Long.MaxValue) + + override def generateConfigs() = { + FixedPortTestUtils.createBrokerConfigs(numServers, + zkConnect, + enableControlledShutdown = false) + .map(KafkaConfig.fromProps(_, overridingProps)) + } + + var producers = mutable.Buffer[KafkaProducer[Array[Byte], Array[Byte]]]() + var consumers = mutable.Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]() + var replicaConsumers = mutable.Buffer[SimpleConsumer]() + + var leaderNode: KafkaServer = null + var followerNode: KafkaServer = null + private val topic1 = "topic-1" + + @Before + override def setUp() { + super.setUp() + val producerProps = new Properties() + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + producerProps.put(ProducerConfig.ACKS_CONFIG, "0") + producerProps.put(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "false") + producerProps.put(ProducerConfig.BUFFER_MEMORY_CONFIG, producerBufferSize.toString) + producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, producerId1) + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, + classOf[org.apache.kafka.common.serialization.ByteArraySerializer]) + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + classOf[org.apache.kafka.common.serialization.ByteArraySerializer]) + producers += new KafkaProducer[Array[Byte], Array[Byte]](producerProps) + + producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, producerId2) + producers += new KafkaProducer[Array[Byte], Array[Byte]](producerProps) + + val numPartitions = 1 + val leaders = TestUtils.createTopic(zkClient, topic1, numPartitions, numServers, servers) + leaderNode = if (leaders(0).get == servers.head.config.brokerId) servers.head else servers(1) + followerNode = if (leaders(0).get != servers.head.config.brokerId) servers.head else servers(1) + assertTrue("Leader of all partitions of the topic should exist", leaders.values.forall(leader => leader.isDefined)) + + // Create consumers + val consumerProps = new Properties + consumerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + consumerProps.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "QuotasTest") + consumerProps.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 4096.toString) + consumerProps.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapUrl) + consumerProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + classOf[org.apache.kafka.common.serialization.ByteArrayDeserializer]) + consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + classOf[org.apache.kafka.common.serialization.ByteArrayDeserializer]) + consumerProps.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "range") + + consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, consumerId1) + consumers += new KafkaConsumer(consumerProps) + // Create replica consumers with the same clientId as the high level consumer. These requests should never be throttled + replicaConsumers += new SimpleConsumer("localhost", leaderNode.boundPort(), 1000000, 64*1024, consumerId1) + + consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, consumerId2) + consumers += new KafkaConsumer(consumerProps) + replicaConsumers += new SimpleConsumer("localhost", leaderNode.boundPort(), 1000000, 64*1024, consumerId2) + + } + + @After + override def tearDown() { + producers.foreach( _.close ) + consumers.foreach( _.close ) + replicaConsumers.foreach( _.close ) + super.tearDown() + } + + @Test + def testThrottledProducerConsumer() { + val allMetrics: mutable.Map[MetricName, KafkaMetric] = leaderNode.metrics.metrics().asScala + + val numRecords = 1000 + produce(producers.head, numRecords) + + val producerMetricName = new MetricName("throttle-time", + RequestKeys.nameForKey(RequestKeys.ProduceKey), + "Tracking throttle-time per client", + "client-id", producerId1) + Assert.assertTrue("Should have been throttled", allMetrics(producerMetricName).value() > 0) + + // Consumer should read in a bursty manner and get throttled immediately + consume(consumers.head, numRecords) + // The replica consumer should not be throttled also. Create a fetch request which will exceed the quota immediately + val request = new FetchRequestBuilder().addFetch(topic1, 0, 0, 1024*1024).replicaId(followerNode.config.brokerId).build() + replicaConsumers.head.fetch(request) + val consumerMetricName = new MetricName("throttle-time", + RequestKeys.nameForKey(RequestKeys.FetchKey), + "Tracking throttle-time per client", + "client-id", consumerId1) + Assert.assertTrue("Should have been throttled", allMetrics(consumerMetricName).value() > 0) + } + + @Test + def testProducerConsumerOverrideUnthrottled() { + val allMetrics: mutable.Map[MetricName, KafkaMetric] = leaderNode.metrics.metrics().asScala + val numRecords = 1000 + produce(producers(1), numRecords) + val producerMetricName = new MetricName("throttle-time", + RequestKeys.nameForKey(RequestKeys.ProduceKey), + "Tracking throttle-time per client", + "client-id", producerId2) + Assert.assertEquals("Should not have been throttled", 0.0, allMetrics(producerMetricName).value()) + + // The "client" consumer does not get throttled. + consume(consumers(1), numRecords) + // The replica consumer should not be throttled also. Create a fetch request which will exceed the quota immediately + val request = new FetchRequestBuilder().addFetch(topic1, 0, 0, 1024*1024).replicaId(followerNode.config.brokerId).build() + replicaConsumers(1).fetch(request) + val consumerMetricName = new MetricName("throttle-time", + RequestKeys.nameForKey(RequestKeys.FetchKey), + "Tracking throttle-time per client", + "client-id", consumerId2) + Assert.assertEquals("Should not have been throttled", 0.0, allMetrics(consumerMetricName).value()) + } + + def produce(p: KafkaProducer[Array[Byte], Array[Byte]], count: Int): Int = { + var numBytesProduced = 0 + for (i <- 0 to count) { + val payload = i.toString.getBytes + numBytesProduced += payload.length + p.send(new ProducerRecord[Array[Byte], Array[Byte]](topic1, null, null, payload), + new ErrorLoggingCallback(topic1, null, null, true)).get() + Thread.sleep(1) + } + numBytesProduced + } + + def consume(consumer: KafkaConsumer[Array[Byte], Array[Byte]], numRecords: Int) { + consumer.subscribe(List(topic1)) + var numConsumed = 0 + while (numConsumed < numRecords) { + for (cr <- consumer.poll(100)) { + numConsumed += 1 + } + } + } +} diff --git a/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala new file mode 100644 index 0000000000000..2f72c782be4df --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala @@ -0,0 +1,231 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES 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.io.File + +import org.apache.kafka.clients.producer.ProducerConfig +import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.clients.producer.KafkaProducer +import org.apache.kafka.clients.consumer.Consumer +import org.apache.kafka.clients.consumer.KafkaConsumer +import org.apache.kafka.clients.consumer.ConsumerRecord +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.common.TopicPartition +import kafka.integration.KafkaServerTestHarness + +import kafka.utils.{TestUtils, Logging} +import kafka.server.KafkaConfig + +import java.util.ArrayList +import org.junit.{Test, Before, After} +import org.junit.Assert._ + +import scala.collection.mutable.Buffer +import scala.collection.JavaConversions._ +import kafka.coordinator.ConsumerCoordinator + + +/** + * Integration tests for the new consumer that cover basic usage as well as server failures + */ +class SSLConsumerTest extends KafkaServerTestHarness with Logging { + + val trustStoreFile = File.createTempFile("truststore", ".jks") + val numServers = 3 + val producerCount = 1 + val consumerCount = 2 + val producerConfig = new Properties + val consumerConfig = new Properties + + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.NumPartitionsProp, 4.toString) + overridingProps.put(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown + overridingProps.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "3") // don't want to lose offset + overridingProps.put(KafkaConfig.OffsetsTopicPartitionsProp, "1") + overridingProps.put(KafkaConfig.ConsumerMinSessionTimeoutMsProp, "100") // set small enough session timeout + + val consumers = Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]() + val producers = Buffer[KafkaProducer[Array[Byte], Array[Byte]]]() + + def generateConfigs() = + TestUtils.createBrokerConfigs(numServers, zkConnect, false, enableSSL=true, trustStoreFile=Some(trustStoreFile)).map(KafkaConfig.fromProps(_, overridingProps)) + + val topic = "topic" + val part = 0 + val tp = new TopicPartition(topic, part) + + // configure the servers and clients + 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") + + @Before + override def setUp() { + super.setUp() + producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getSSLBrokerListStrFromServers(servers)) + 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, TestUtils.getSSLBrokerListStrFromServers(servers)) + 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 += TestUtils.createNewProducer(TestUtils.getSSLBrokerListStrFromServers(servers), + acks = 1, + enableSSL=true, + trustStoreFile=Some(trustStoreFile)) + for (i <- 0 until consumerCount) + consumers += TestUtils.createNewConsumer(TestUtils.getSSLBrokerListStrFromServers(servers), + groupId = "my-test", + partitionAssignmentStrategy= "range", + enableSSL=true, + trustStoreFile=Some(trustStoreFile)) + + + // create the consumer offset topic + TestUtils.createTopic(zkClient, ConsumerCoordinator.OffsetsTopicName, + overridingProps.getProperty(KafkaConfig.OffsetsTopicPartitionsProp).toInt, + overridingProps.getProperty(KafkaConfig.OffsetsTopicReplicationFactorProp).toInt, + servers, + servers(0).consumerCoordinator.offsetsTopicConfigs) + + // create the test topic with all the brokers as replicas + TestUtils.createTopic(this.zkClient, topic, 1, numServers, this.servers) + } + + @After + override def tearDown() { + producers.foreach(_.close()) + consumers.foreach(_.close()) + super.tearDown() + } + + @Test + def testSimpleConsumption() { + val numRecords = 10000 + sendRecords(numRecords) + assertEquals(0, this.consumers(0).assignment.size) + this.consumers(0).assign(List(tp)) + assertEquals(1, this.consumers(0).assignment.size) + this.consumers(0).seek(tp, 0) + consumeRecords(this.consumers(0), numRecords = numRecords, startingOffset = 0) + } + + @Test + def testAutoOffsetReset() { + sendRecords(1) + this.consumers(0).assign(List(tp)) + consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0) + } + + @Test + def testSeek() { + val consumer = this.consumers(0) + val totalRecords = 50L + sendRecords(totalRecords.toInt) + consumer.assign(List(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) + } + + @Test + def testGroupConsumption() { + sendRecords(10) + this.consumers(0).subscribe(List(topic)) + consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0) + } + + @Test + def testPositionAndCommit() { + sendRecords(5) + + // committed() on a partition with no committed offset returns null + assertNull(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).assign(List(tp)) + + assertEquals("position() on a partition that we are subscribed to should reset the offset", 0L, this.consumers(0).position(tp)) + this.consumers(0).commitSync() + assertEquals(0L, this.consumers(0).committed(tp).offset) + + consumeRecords(this.consumers(0), 5, 0) + assertEquals("After consuming 5 records, position should be 5", 5L, this.consumers(0).position(tp)) + this.consumers(0).commitSync() + assertEquals("Committed offset should be returned", 5L, this.consumers(0).committed(tp).offset) + + sendRecords(1) + + // another consumer in the same group should get the same position + this.consumers(1).assign(List(tp)) + consumeRecords(this.consumers(1), 1, 5) + } + + @Test + 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")) + } + + 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) + } + + 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()) + } + } + +} diff --git a/core/src/test/scala/integration/kafka/api/SSLProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/SSLProducerSendTest.scala new file mode 100644 index 0000000000000..0f706248625d4 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/SSLProducerSendTest.scala @@ -0,0 +1,240 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 java.io.File + +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.{After, Before, Test} + + +class SSLProducerSendTest extends KafkaServerTestHarness { + val numServers = 2 + val trustStoreFile = File.createTempFile("truststore", ".jks") + + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.NumPartitionsProp, 4.toString) + + def generateConfigs() = + TestUtils.createBrokerConfigs(numServers, zkConnect, false, enableSSL=true, trustStoreFile=Some(trustStoreFile)).map(KafkaConfig.fromProps(_, overridingProps)) + + private var consumer1: SimpleConsumer = null + private var consumer2: SimpleConsumer = null + + private val topic = "topic" + private val numRecords = 100 + + @Before + 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, "") + + } + + @After + 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 sslProducer = TestUtils.createNewProducer(TestUtils.getSSLBrokerListStrFromServers(servers), enableSSL=true, trustStoreFile=Some(trustStoreFile)) + var producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers)) + 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, sslProducer.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, sslProducer.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, sslProducer.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, sslProducer.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) + sslProducer.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 with sslProducer + for (i <- 1 to numRecords) + sslProducer.send(record0, callback) + // check that all messages have been acked via offset + assertEquals("Should have offset " + numRecords + 4L, numRecords + 4L, sslProducer.send(record0, callback).get.offset) + + //non-blocking send a list of records with plaintext producer + for (i <- 1 to numRecords) + producer.send(record0, callback) + + // check that all messages have been acked via offset + assertEquals("Should have offset " + (numRecords * 2 + 5L), numRecords * 2 + 5L, producer.send(record0, callback).get.offset) + + } finally { + if (sslProducer != null) { + sslProducer.close() + sslProducer = null + } + if (producer != null) { + producer.close() + producer = null + } + + } + } + + /** + * 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(TestUtils.getSSLBrokerListStrFromServers(servers), enableSSL=true, trustStoreFile=Some(trustStoreFile)) + 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(TestUtils.getSSLBrokerListStrFromServers(servers), enableSSL=true, trustStoreFile=Some(trustStoreFile)) + 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 + } + } + } +} 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/main/scala/kafka/tools/TestLogCleaning.scala b/core/src/test/scala/kafka/tools/TestLogCleaning.scala old mode 100644 new mode 100755 similarity index 90% rename from core/src/main/scala/kafka/tools/TestLogCleaning.scala rename to core/src/test/scala/kafka/tools/TestLogCleaning.scala index 1d4ea93f2ba8d..dcbfbe1c1fc14 --- a/core/src/main/scala/kafka/tools/TestLogCleaning.scala +++ b/core/src/test/scala/kafka/tools/TestLogCleaning.scala @@ -52,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") @@ -84,38 +89,39 @@ 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) } - + 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 @@ -132,12 +138,12 @@ object TestLogCleaning { 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)) } } @@ -169,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() @@ -233,16 +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(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "true") producerProps.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerUrl) - val producer = new KafkaProducer(producerProps) + 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") @@ -254,9 +264,9 @@ object TestLogCleaning { val delete = i % 100 < percentDeletes val msg = if(delete) - new ProducerRecord(topic, key.toString.getBytes(), null) + new ProducerRecord[Array[Byte],Array[Byte]](topic, key.toString.getBytes(), null) else - new ProducerRecord(topic, key.toString.getBytes(), i.toString.getBytes()) + 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() 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 e19b8b2838355..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) } }) 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/TestLinearWriteSpeed.scala b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala old mode 100644 new mode 100755 index 7211c2529c1db..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._ @@ -110,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) @@ -193,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) @@ -201,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 index 41f334d48897b..e0e46c8838f71 100644 --- a/core/src/test/scala/other/kafka/TestOffsetManager.scala +++ b/core/src/test/scala/other/kafka/TestOffsetManager.scala @@ -1,8 +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. + */ + package other.kafka import org.I0Itec.zkclient.ZkClient import kafka.api._ -import kafka.utils.{ShutdownableThread, ZKStringSerializer} +import kafka.utils.{ZkUtils, ShutdownableThread} +import org.apache.kafka.common.protocol.SecurityProtocol import scala.collection._ import kafka.client.ClientUtils import joptsimple.OptionParser @@ -12,7 +30,6 @@ import scala.util.Random import java.io.IOException import kafka.metrics.{KafkaTimer, KafkaMetricsGroup} import java.util.concurrent.TimeUnit -import com.yammer.metrics.core.Gauge import java.util.concurrent.atomic.AtomicInteger import java.nio.channels.ClosedByInterruptException @@ -50,9 +67,9 @@ object TestOffsetManager { extends ShutdownableThread("commit-thread") with KafkaMetricsGroup { - private val group = "group-" + id + private val groupId = "group-" + id private val metadata = "Metadata from commit thread " + id - private var offsetsChannel = ClientUtils.channelToOffsetManager(group, zkClient, SocketTimeoutMs) + private var offsetsChannel = ClientUtils.channelToOffsetManager(groupId, zkClient, SocketTimeoutMs) private var offset = 0L val numErrors = new AtomicInteger(0) val numCommits = new AtomicInteger(0) @@ -62,17 +79,17 @@ object TestOffsetManager { private def ensureConnected() { if (!offsetsChannel.isConnected) - offsetsChannel = ClientUtils.channelToOffsetManager(group, zkClient, SocketTimeoutMs) + offsetsChannel = ClientUtils.channelToOffsetManager(groupId, zkClient, SocketTimeoutMs) } override def doWork() { - val commitRequest = OffsetCommitRequest(group, immutable.Map((1 to partitionCount).map(TopicAndPartition("topic-" + id, _) -> OffsetAndMetadata(offset, metadata)):_*)) + 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().buffer) + val response = OffsetCommitResponse.readFrom(offsetsChannel.receive().payload()) if (response.commitStatus.exists(_._2 != ErrorMapping.NoError)) numErrors.getAndIncrement } offset += 1 @@ -81,7 +98,7 @@ object TestOffsetManager { 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, group, e2)) + 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 { @@ -119,7 +136,7 @@ object TestOffsetManager { val group = "group-" + id try { metadataChannel.send(ConsumerMetadataRequest(group)) - val coordinatorId = ConsumerMetadataResponse.readFrom(metadataChannel.receive().buffer).coordinatorOpt.map(_.id).getOrElse(-1) + val coordinatorId = ConsumerMetadataResponse.readFrom(metadataChannel.receive().payload()).coordinatorOpt.map(_.id).getOrElse(-1) val channel = if (channels.contains(coordinatorId)) channels(coordinatorId) @@ -135,7 +152,7 @@ object TestOffsetManager { channel.send(fetchRequest) fetchTimer.time { - val response = OffsetFetchResponse.readFrom(channel.receive().buffer) + val response = OffsetFetchResponse.readFrom(channel.receive().payload()) if (response.requestInfo.exists(_._2.error != ErrorMapping.NoError)) { numErrors.getAndIncrement } @@ -238,7 +255,7 @@ object TestOffsetManager { var fetchThread: FetchThread = null var statsThread: StatsThread = null try { - zkClient = new ZkClient(zookeeper, 6000, 2000, ZKStringSerializer) + zkClient = ZkUtils.createZkClient(zookeeper, 6000, 2000) commitThreads = (0 to (threadCount-1)).map { threadId => new CommitThread(threadId, partitionCount, commitIntervalMs, zkClient) } 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/unit/kafka/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala new file mode 100644 index 0000000000000..1233104bc52af --- /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 org.junit.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/AclCommandTest.scala b/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala new file mode 100644 index 0000000000000..e2a75e20a8fa5 --- /dev/null +++ b/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala @@ -0,0 +1,134 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package unit.kafka.admin + +import java.io.StringReader +import java.util.Properties + +import kafka.admin.AclCommand +import kafka.security.auth._ +import kafka.server.KafkaConfig +import kafka.utils.{Logging, TestUtils} +import kafka.zk.ZooKeeperTestHarness +import org.apache.kafka.common.security.auth.KafkaPrincipal +import org.junit.{Assert, Test} + +class AclCommandTest extends ZooKeeperTestHarness with Logging { + + private val Users = Set(KafkaPrincipal.fromString("User:test1"), KafkaPrincipal.fromString("User:test2")) + private val UsersString = Users.mkString(AclCommand.Delimiter.toString) + private val Hosts = Set("host1", "host2") + private val HostsString = Hosts.mkString(AclCommand.Delimiter.toString) + + private val TopicResources = Set(new Resource(Topic, "test-1"), new Resource(Topic, "test-2")) + private val ConsumerGroupResources = Set(new Resource(ConsumerGroup, "testGroup-1"), new Resource(ConsumerGroup, "testGroup-2")) + + private val ResourceToCommand = Map[Set[Resource], Array[String]]( + TopicResources -> Array("--topic", "test-1,test-2"), + Set(Resource.ClusterResource) -> Array("--cluster"), + ConsumerGroupResources -> Array("--consumer-group", "testGroup-1,testGroup-2") + ) + + private val ResourceToOperations = Map[Set[Resource], (Set[Operation], Array[String])]( + TopicResources -> (Set(Read, Write, Describe), Array("--operations", "Read,Write,Describe")), + Set(Resource.ClusterResource) -> (Set(Create, ClusterAction), Array("--operations", "Create,ClusterAction")), + ConsumerGroupResources -> (Set(Read).toSet[Operation], Array("--operations", "Read")) + ) + + private val ProducerResourceToAcls = Map[Set[Resource], Set[Acl]]( + TopicResources -> AclCommand.getAcls(Users, Allow, Set(Write, Describe), Hosts), + Set(Resource.ClusterResource) -> AclCommand.getAcls(Users, Allow, Set(Create), Hosts) + ) + + private val ConsumerResourceToAcls = Map[Set[Resource], Set[Acl]]( + TopicResources -> AclCommand.getAcls(Users, Allow, Set(Read, Describe), Hosts), + ConsumerGroupResources -> AclCommand.getAcls(Users, Allow, Set(Read), Hosts) + ) + + private val CmdToResourcesToAcl = Map[Array[String], Map[Set[Resource], Set[Acl]]]( + Array[String]("--producer") -> ProducerResourceToAcls, + Array[String]("--consumer") -> ConsumerResourceToAcls, + Array[String]("--producer", "--consumer") -> ConsumerResourceToAcls.map { case (k, v) => k -> (v ++ + ProducerResourceToAcls.getOrElse(k, Set.empty[Acl])) } + ) + + @Test + def testAclCli() { + val brokerProps = TestUtils.createBrokerConfig(0, zkConnect) + brokerProps.put(KafkaConfig.AuthorizerClassNameProp, "kafka.security.auth.SimpleAclAuthorizer") + val args = Array("--authorizer-properties", "zookeeper.connect=" + zkConnect) + + for ((resources, resourceCmd) <- ResourceToCommand) { + for (permissionType <- PermissionType.values) { + val operationToCmd = ResourceToOperations(resources) + val (acls, cmd) = getAclToCommand(permissionType, operationToCmd._1) + AclCommand.main(args ++ cmd ++ resourceCmd ++ operationToCmd._2 :+ "--add") + for (resource <- resources) { + TestUtils.waitAndVerifyAcls(acls, getAuthorizer(brokerProps), resource) + } + + testRemove(resources, resourceCmd, args, brokerProps) + } + } + } + + @Test + def testProducerConsumerCli() { + val brokerProps = TestUtils.createBrokerConfig(0, zkConnect) + brokerProps.put(KafkaConfig.AuthorizerClassNameProp, "kafka.security.auth.SimpleAclAuthorizer") + val args = Array("--authorizer-properties", "zookeeper.connect=" + zkConnect) + + for ((cmd, resourcesToAcls) <- CmdToResourcesToAcl) { + val resourceCommand: Array[String] = resourcesToAcls.keys.map(ResourceToCommand).foldLeft(Array[String]())(_ ++ _) + AclCommand.main(args ++ getCmd(Allow) ++ resourceCommand ++ cmd :+ "--add") + for ((resources, acls) <- resourcesToAcls) { + for (resource <- resources) { + TestUtils.waitAndVerifyAcls(acls, getAuthorizer(brokerProps), resource) + } + } + testRemove(resourcesToAcls.keys.flatten.toSet, resourceCommand, args, brokerProps) + } + } + + private def testRemove(resources: Set[Resource], resourceCmd: Array[String], args: Array[String], brokerProps: Properties) { + for (resource <- resources) { + Console.withIn(new StringReader(s"y${AclCommand.Newline}" * resources.size)) { + AclCommand.main(args ++ resourceCmd :+ "--remove") + TestUtils.waitAndVerifyAcls(Set.empty[Acl], getAuthorizer(brokerProps), resource) + } + } + } + + private def getAclToCommand(permissionType: PermissionType, operations: Set[Operation]): (Set[Acl], Array[String]) = { + (AclCommand.getAcls(Users, permissionType, operations, Hosts), getCmd(permissionType)) + } + + private def getCmd(permissionType: PermissionType): Array[String] = { + if (permissionType == Allow) + Array("--allow-principals", UsersString, "--allow-hosts", HostsString) + else + Array("--deny-principals", UsersString, "--deny-hosts", HostsString) + } + + def getAuthorizer(props: Properties): Authorizer = { + val kafkaConfig = KafkaConfig.fromProps(props) + val authZ = new SimpleAclAuthorizer + authZ.configure(kafkaConfig.originals) + + authZ + } +} 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 1bf2667f47853..ed940393d4a97 --- 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.scalatest.junit.JUnit3Suite +import org.junit.Assert._ +import org.apache.kafka.common.protocol.SecurityProtocol 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} +import org.junit.{Test, After, Before} -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, false) - val configProps2 = TestUtils.createBrokerConfig(brokerId2, port2, false) - val configProps3 = TestUtils.createBrokerConfig(brokerId3, port3, false) - val configProps4 = TestUtils.createBrokerConfig(brokerId4, port4, false) - +class AddPartitionsTest extends ZooKeeperTestHarness { + var configs: Seq[KafkaConfig] = null var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] var brokers: Seq[Broker] = Seq.empty[Broker] @@ -52,16 +39,14 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { val topic3 = "new-topic3" val topic4 = "new-topic4" + @Before 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)) + 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) @@ -70,12 +55,14 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { createTopic(zkClient, topic4, partitionReplicaAssignment = Map(0->Seq(0,3)), servers = servers) } + @After 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() } + @Test def testTopicDoesNotExist { try { AdminUtils.addPartitions(zkClient, "Blah", 1) @@ -86,6 +73,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { } } + @Test def testWrongReplicaCount { try { AdminUtils.addPartitions(zkClient, topic1, 2, "0:1,0:1:2") @@ -96,6 +84,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { } } + @Test def testIncrementPartitions { AdminUtils.addPartitions(zkClient, topic1, 3) // wait until leader is elected @@ -109,10 +98,10 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { // read metadata from a broker and verify the new topic partitions exist TestUtils.waitUntilMetadataIsPropagated(servers, topic1, 1) TestUtils.waitUntilMetadataIsPropagated(servers, topic1, 2) - val metadata = ClientUtils.fetchTopicMetadata(Set(topic1), brokers, "AddPartitionsTest-testIncrementPartitions", + 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) @@ -121,6 +110,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { assert(replicas.contains(partitionDataForTopic1(1).leader.get)) } + @Test def testManualAssignmentOfReplicas { AdminUtils.addPartitions(zkClient, topic2, 3, "1:2,0:1,2:3") // wait until leader is elected @@ -134,10 +124,10 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { // read metadata from a broker and verify the new topic partitions exist TestUtils.waitUntilMetadataIsPropagated(servers, topic2, 1) TestUtils.waitUntilMetadataIsPropagated(servers, topic2, 2) - val metadata = ClientUtils.fetchTopicMetadata(Set(topic2), brokers, "AddPartitionsTest-testManualAssignmentOfReplicas", + 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) @@ -147,6 +137,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { assert(replicas(1).id == 0 || replicas(1).id == 1) } + @Test def testReplicaPlacement { AdminUtils.addPartitions(zkClient, topic3, 7) @@ -158,16 +149,17 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 5) TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 6) - val metadata = ClientUtils.fetchTopicMetadata(Set(topic3), brokers, "AddPartitionsTest-testReplicaPlacement", + 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) 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 e28979827110d..2d18069884d18 --- a/core/src/test/scala/unit/kafka/admin/AdminTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AdminTest.scala @@ -18,19 +18,18 @@ package kafka.admin import junit.framework.Assert._ import org.junit.Test -import org.scalatest.junit.JUnit3Suite import java.util.Properties import kafka.utils._ import kafka.log._ import kafka.zk.ZooKeeperTestHarness import kafka.utils.{Logging, ZkUtils, TestUtils} -import kafka.common.{TopicExistsException, TopicAndPartition} -import kafka.server.{KafkaServer, KafkaConfig} +import kafka.common.{InvalidTopicException, TopicExistsException, TopicAndPartition} +import kafka.server.{ConfigType, KafkaServer, KafkaConfig} import java.io.File import TestUtils._ -class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { +class AdminTest extends ZooKeeperTestHarness with Logging { @Test def testReplicaAssignment() { @@ -67,7 +66,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { @Test def testManualReplicaAssignment() { val brokers = List(0, 1, 2, 3, 4) - TestUtils.createBrokersInZk(zkClient, brokers) + TestUtils.createBrokersInZk(zkClient, zkConnection, brokers) // duplicate brokers intercept[IllegalArgumentException] { @@ -118,7 +117,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { 11 -> 1 ) val topic = "test" - TestUtils.createBrokersInZk(zkClient, List(0, 1, 2, 3, 4)) + TestUtils.createBrokersInZk(zkClient, zkConnection, List(0, 1, 2, 3, 4)) // create the topic AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // create leaders for all partitions @@ -134,6 +133,20 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { } } + @Test + def testTopicCreationWithCollision() { + val topic = "test.topic" + val collidingTopic = "test_topic" + TestUtils.createBrokersInZk(zkClient, zkConnection, List(0, 1, 2, 3, 4)) + // create the topic + AdminUtils.createTopic(zkClient, topic, 3, 1) + + intercept[InvalidTopicException] { + // shouldn't be able to create a topic that collides + AdminUtils.createTopic(zkClient, collidingTopic, 3, 1) + } + } + private def getBrokersWithPartitionDir(servers: Iterable[KafkaServer], topic: String, partitionId: Int): Set[Int] = { servers.filter(server => new File(server.config.logDirs.head, topic + "-" + partitionId).exists) .map(_.config.brokerId) @@ -145,7 +158,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, false).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 @@ -176,7 +189,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, false).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 @@ -207,7 +220,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, false).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 @@ -236,13 +249,13 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { def testReassigningNonExistingPartition() { val topic = "test" // create brokers - val servers = TestUtils.createBrokerConfigs(4, false).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()) @@ -262,7 +275,7 @@ 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, false).map(b => TestUtils.createServer(new KafkaConfig(b))) + val servers = TestUtils.createBrokerConfigs(2, zkConnect, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) // wait until reassignment completes TestUtils.waitUntilTrue(() => !checkIfReassignPartitionPathExists(zkClient), @@ -298,10 +311,10 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val partition = 1 val preferredReplica = 0 // create brokers - val serverConfigs = TestUtils.createBrokerConfigs(3, false).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, oldLeaderOpt = None).get // trigger preferred replica election @@ -318,8 +331,8 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val topic = "test" val partition = 1 // create brokers - val serverConfigs = TestUtils.createBrokerConfigs(3, false).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 TestUtils.createTopic(zkClient, topic, partitionReplicaAssignment = expectedReplicaAssignment, servers = servers) @@ -330,7 +343,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { try { // wait for the update metadata request to trickle to the brokers TestUtils.waitUntilTrue(() => - activeServers.foldLeft(true)(_ && _.apis.metadataCache.getPartitionInfo(topic,partition).get.leaderIsrAndControllerEpoch.leaderAndIsr.isr.size != 3), + 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.getPartitionInfo(topic,partition).get @@ -346,11 +359,11 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { leaderAfterShutdown = partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leader assertEquals(0, leaderAfterShutdown) - assertTrue(servers.foldLeft(true)(_ && _.apis.metadataCache.getPartitionInfo(topic,partition).get.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.getPartitionInfo(topic,partition).get.leaderIsrAndControllerEpoch.leaderAndIsr.leader == 0)) + assertTrue(servers.forall(_.apis.metadataCache.getPartitionInfo(topic,partition).get.leaderIsrAndControllerEpoch.leaderAndIsr.leader == 0)) } finally { servers.foreach(_.shutdown()) @@ -365,12 +378,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 } @@ -393,12 +406,16 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { checkConfig(maxMessageSize, retentionMs) // now double the config values for the topic and check that it is applied + val newConfig: Properties = makeConfig(2*maxMessageSize, 2 * retentionMs) AdminUtils.changeTopicConfig(server.zkClient, topic, makeConfig(2*maxMessageSize, 2 * retentionMs)) checkConfig(2*maxMessageSize, 2 * retentionMs) + + // Verify that the same config can be read from ZK + val configInZk = AdminUtils.fetchEntityConfig(server.zkClient, ConfigType.Topic, topic) + assertEquals(newConfig, configInZk) } 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/ConfigCommandTest.scala b/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala new file mode 100644 index 0000000000000..0379064a360a8 --- /dev/null +++ b/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala @@ -0,0 +1,68 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.admin + +import junit.framework.Assert._ +import kafka.admin.ConfigCommand.ConfigCommandOptions +import org.junit.Test +import kafka.utils.Logging +import kafka.zk.ZooKeeperTestHarness + +class ConfigCommandTest extends ZooKeeperTestHarness with Logging { + @Test + def testArgumentParse() { + // Should parse correctly + var createOpts = new ConfigCommandOptions(Array("--zookeeper", zkConnect, + "--entity-name", "x", + "--entity-type", "clients", + "--describe")) + createOpts.checkArgs() + + // For --alter and added config + createOpts = new ConfigCommandOptions(Array("--zookeeper", zkConnect, + "--entity-name", "x", + "--entity-type", "clients", + "--alter", + "--added-config", "a=b,c=d")) + createOpts.checkArgs() + + // For alter and deleted config + createOpts = new ConfigCommandOptions(Array("--zookeeper", zkConnect, + "--entity-name", "x", + "--entity-type", "clients", + "--alter", + "--deleted-config", "a,b,c")) + createOpts.checkArgs() + + // For alter and both added, deleted config + createOpts = new ConfigCommandOptions(Array("--zookeeper", zkConnect, + "--entity-name", "x", + "--entity-type", "clients", + "--alter", + "--added-config", "a=b,c=d", + "--deleted-config", "a")) + createOpts.checkArgs() + val addedProps = ConfigCommand.parseConfigsToBeAdded(createOpts) + assertEquals(2, addedProps.size()) + assertEquals("b", addedProps.getProperty("a")) + assertEquals("d", addedProps.getProperty("c")) + + val deletedProps = ConfigCommand.parseConfigsToBeDeleted(createOpts) + assertEquals(1, deletedProps.size) + assertEquals("a", deletedProps(0)) + } +} \ No newline at end of file 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..d3abf0884bc8f --- /dev/null +++ b/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala @@ -0,0 +1,211 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 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 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 29cc01bcef9ca..fbae398a3bb4e 100644 --- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala @@ -16,22 +16,16 @@ */ package kafka.admin -import org.scalatest.junit.JUnit3Suite +import kafka.log.Log 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 +import kafka.common.{TopicAlreadyMarkedForDeletionException, TopicAndPartition} -class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { +class DeleteTopicTest extends ZooKeeperTestHarness { @Test def testDeleteTopicWithAllAliveReplicas() { @@ -40,7 +34,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()) } @@ -59,13 +53,13 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { // check if all replicas but the one that is shut down has deleted the log TestUtils.waitUntilTrue(() => servers.filter(s => s.config.brokerId != follower.config.brokerId) - .foldLeft(true)((res, server) => res && server.getLogManager().getLog(topicAndPartition).isEmpty), "Replicas 0,1 have not deleted log.") + .forall(_.getLogManager().getLog(topicAndPartition).isEmpty), "Replicas 0,1 have not deleted log.") // ensure topic deletion is halted 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()) } @@ -92,7 +86,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { controller.startup() follower.startup() - verifyTopicDeletion(topic, servers) + TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) servers.foreach(_.shutdown()) } @@ -101,16 +95,15 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) val topic = "test" val topicAndPartition = TopicAndPartition(topic, 0) - val brokerConfigs = TestUtils.createBrokerConfigs(4, false) + val brokerConfigs = TestUtils.createBrokerConfigs(4, zkConnect, false) brokerConfigs.foreach(p => p.setProperty("delete.topic.enable", "true")) // create brokers - val allServers = brokerConfigs.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 - TestUtils.waitUntilTrue(() => servers.foldLeft(true)((res, server) => - res && server.getLogManager().getLog(topicAndPartition).isDefined), + 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) @@ -138,7 +131,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, 0) assertEquals("Partition should not be reassigned to 0, 1, 2", oldAssignedReplicas, assignedReplicas) follower.startup() - verifyTopicDeletion(topic, servers) + TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) allServers.foreach(_.shutdown()) } @@ -157,10 +150,10 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { 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 TestUtils.waitUntilTrue(() => - servers.foldLeft(true)((res, server) => res && server.getLogManager().getLog(newPartition).isEmpty), + servers.forall(_.getLogManager().getLog(newPartition).isEmpty), "Replica logs not for new partition [test,1] not deleted after delete topic is complete.") servers.foreach(_.shutdown()) } @@ -175,10 +168,10 @@ 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()) } @@ -190,14 +183,14 @@ 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) // 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 - TestUtils.waitUntilTrue(() => servers.foldLeft(true)((res, server) => res && server.getLogManager().getLog(topicAndPartition).isDefined), + TestUtils.waitUntilTrue(() => servers.forall(_.getLogManager().getLog(topicAndPartition).isDefined), "Replicas for topic test not created.") servers.foreach(_.shutdown()) } @@ -210,10 +203,9 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { // start topic deletion AdminUtils.deleteTopic(zkClient, "test2") // verify delete topic path for test2 is removed from zookeeper - verifyTopicDeletion("test2", servers) + TestUtils.verifyTopicDeletion(zkClient, "test2", 1, servers) // verify that topic test is untouched - TestUtils.waitUntilTrue(() => servers.foldLeft(true)((res, server) => - res && server.getLogManager().getLog(topicAndPartition).isDefined), + 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))) @@ -221,37 +213,90 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { 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()) + } + @Test + def testDeleteTopicAlreadyMarkedAsDeleted() { + val topicAndPartition = TopicAndPartition("test", 0) + val topic = topicAndPartition.topic + val servers = createTestTopicAndCluster(topic) + + try { + // start topic deletion + AdminUtils.deleteTopic(zkClient, topic) + // try to delete topic marked as deleted + AdminUtils.deleteTopic(zkClient, topic) + fail("Expected TopicAlreadyMarkedForDeletionException") + } + catch { + case e: TopicAlreadyMarkedForDeletionException => // expected exception + } + + TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) + servers.foreach(_.shutdown()) } private def createTestTopicAndCluster(topic: String): Seq[KafkaServer] = { + + 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) - val brokerConfigs = TestUtils.createBrokerConfigs(3, false) - brokerConfigs.foreach(p => p.setProperty("delete.topic.enable", "true")) // create brokers - val servers = brokerConfigs.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 - TestUtils.waitUntilTrue(() => servers.foldLeft(true)((res, server) => - res && server.getLogManager().getLog(topicAndPartition).isDefined), + 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 - TestUtils.waitUntilTrue(() => !ZkUtils.pathExists(zkClient, ZkUtils.getDeleteTopicPath(topic)), - "Admin path /admin/delete_topic/test path not deleted even after a replica is restarted") - TestUtils.waitUntilTrue(() => !ZkUtils.pathExists(zkClient, ZkUtils.getTopicPath(topic)), - "Topic path /brokers/topics/test not deleted after /admin/delete_topic/test path is deleted") - // ensure that the topic-partition has been deleted from all brokers' replica managers - TestUtils.waitUntilTrue(() => servers.foldLeft(true)((res, server) => res && server.replicaManager.getPartition(topic, 0) == 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.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) + } } } diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala index ac6dd2087de45..d4fa0d5c1b58a 100644 --- a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala @@ -18,15 +18,15 @@ 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.server.KafkaConfig +import kafka.server.ConfigType import kafka.admin.TopicCommand.TopicCommandOptions import kafka.utils.ZkUtils +import kafka.coordinator.ConsumerCoordinator -class TopicCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { +class TopicCommandTest extends ZooKeeperTestHarness with Logging { @Test def testConfigPreservationAcrossPartitionAlteration() { @@ -36,28 +36,65 @@ class TopicCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Loggin val cleanupVal = "compact" // create brokers val brokers = List(0, 1, 2) - TestUtils.createBrokersInZk(zkClient, brokers) + TestUtils.createBrokersInZk(zkClient, zkConnection, 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) + val props = AdminUtils.fetchEntityConfig(zkClient, ConfigType.Topic, 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) + ZkUtils.createPersistentPath(zkClient, ZkUtils.EntityConfigChangesPath) // modify the topic to add new partitions val numPartitionsModified = 3 - val alterOpts = new TopicCommandOptions(Array("--partitions", numPartitionsModified.toString, - "--config", cleanupKey + "=" + cleanupVal, - "--topic", topic)) + val alterOpts = new TopicCommandOptions(Array("--partitions", numPartitionsModified.toString, "--topic", topic)) TopicCommand.alterTopic(zkClient, alterOpts) - val newProps = AdminUtils.fetchTopicConfig(zkClient, topic) + val newProps = AdminUtils.fetchEntityConfig(zkClient, ConfigType.Topic, topic) assertTrue("Updated properties do not contain " + cleanupKey, newProps.containsKey(cleanupKey)) assertTrue("Updated properties have incorrect value", newProps.getProperty(cleanupKey).equals(cleanupVal)) } -} \ No newline at end of file + + @Test + def testTopicDeletion() { + val normalTopic = "test" + + val numPartitionsOriginal = 1 + + // create brokers + val brokers = List(0, 1, 2) + TestUtils.createBrokersInZk(zkClient, zkConnection, 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)) + } +} diff --git a/core/src/test/scala/unit/kafka/api/ApiUtilsTest.scala b/core/src/test/scala/unit/kafka/api/ApiUtilsTest.scala index 255442526d941..fff3e7b87d3e0 100644 --- a/core/src/test/scala/unit/kafka/api/ApiUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/api/ApiUtilsTest.scala @@ -19,7 +19,7 @@ package kafka.api import org.junit._ import org.scalatest.junit.JUnitSuite -import junit.framework.Assert._ +import org.junit.Assert._ import scala.util.Random import java.nio.ByteBuffer import kafka.common.KafkaException diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index cd16ced5465d0..b7e7967c07b7f 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -17,20 +17,25 @@ package kafka.api -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 java.nio.channels.GatheringByteChannel + +import kafka.cluster.{BrokerEndPoint, EndPoint, Broker} import kafka.common.{OffsetAndMetadata, ErrorMapping, OffsetMetadataAndError} +import kafka.common._ +import kafka.consumer.FetchRequestAndResponseStatsRegistry +import kafka.message.{Message, ByteBufferMessageSet} import kafka.utils.SystemTime -import org.apache.kafka.common.requests._ -import org.apache.kafka.common.protocol.ApiKeys -import scala.Some + import kafka.controller.LeaderIsrAndControllerEpoch import kafka.common.TopicAndPartition -import org.apache.kafka.common.TopicPartition + +import java.nio.ByteBuffer + +import org.apache.kafka.common.protocol.SecurityProtocol +import org.junit._ +import org.scalatest.junit.JUnitSuite +import org.junit.Assert._ object SerializationTestUtils { @@ -59,7 +64,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 { @@ -80,21 +85,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 = { @@ -122,7 +153,7 @@ object SerializationTestUtils { ProducerResponse(1, Map( TopicAndPartition(topic1, 0) -> ProducerResponseStatus(0.toShort, 10001), TopicAndPartition(topic2, 0) -> ProducerResponseStatus(0.toShort, 20001) - )) + ), ProducerRequest.CurrentVersion, 100) def createTestFetchRequest: FetchRequest = { new FetchRequest(requestInfo = requestInfos) @@ -148,13 +179,26 @@ object SerializationTestUtils { } def createTestTopicMetadataResponse: TopicMetadataResponse = { - new TopicMetadataResponse(brokers, Seq(topicmetaData1, topicmetaData2), 1) + new TopicMetadataResponse(brokers.map(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)).toSeq, Seq(topicmetaData1, topicmetaData2), 1) + } + + 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 createTestOffsetCommitRequestV1: OffsetCommitRequest = { - new OffsetCommitRequest("group 1", collection.immutable.Map( - TopicAndPartition(topic1, 0) -> OffsetAndMetadata(offset=42L, metadata="some metadata", timestamp=SystemTime.milliseconds), - TopicAndPartition(topic1, 1) -> OffsetAndMetadata(offset=100L, metadata=OffsetAndMetadata.NoMetadata, timestamp=SystemTime.milliseconds) + 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) )) } @@ -163,8 +207,8 @@ object SerializationTestUtils { versionId = 0, groupId = "group 1", requestInfo = collection.immutable.Map( - TopicAndPartition(topic1, 0) -> OffsetAndMetadata(offset=42L, metadata="some metadata", timestamp=SystemTime.milliseconds), - TopicAndPartition(topic1, 1) -> OffsetAndMetadata(offset=100L, metadata=OffsetAndMetadata.NoMetadata, timestamp=SystemTime.milliseconds) + TopicAndPartition(topic1, 0) -> OffsetAndMetadata(42L, "some metadata"), + TopicAndPartition(topic1, 1) -> OffsetAndMetadata(100L, OffsetMetadata.NoMetadata) )) } @@ -183,7 +227,7 @@ object SerializationTestUtils { def createTestOffsetFetchResponse: OffsetFetchResponse = { new OffsetFetchResponse(collection.immutable.Map( TopicAndPartition(topic1, 0) -> OffsetMetadataAndError(42L, "some metadata", ErrorMapping.NoError), - TopicAndPartition(topic1, 1) -> OffsetMetadataAndError(100L, OffsetAndMetadata.NoMetadata, ErrorMapping.UnknownTopicOrPartitionCode) + TopicAndPartition(topic1, 1) -> OffsetMetadataAndError(100L, OffsetMetadata.NoMetadata, ErrorMapping.UnknownTopicOrPartitionCode) )) } @@ -192,29 +236,23 @@ object SerializationTestUtils { } def createConsumerMetadataResponse: ConsumerMetadataResponse = { - ConsumerMetadataResponse(Some(brokers.head), ErrorMapping.NoError) + ConsumerMetadataResponse(Some(brokers.head.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)), ErrorMapping.NoError, 0) } - def createHeartbeatRequestAndHeader: HeartbeatRequestAndHeader = { - val body = new HeartbeatRequest("group1", 1, "consumer1") - HeartbeatRequestAndHeader(0.asInstanceOf[Short], 1, "", body) - } - - def createHeartbeatResponseAndHeader: HeartbeatResponseAndHeader = { - val body = new HeartbeatResponse(0.asInstanceOf[Short]) - HeartbeatResponseAndHeader(1, body) - } - - def createJoinGroupRequestAndHeader: JoinGroupRequestAndHeader = { - import scala.collection.JavaConversions._ - val body = new JoinGroupRequest("group1", 30000, List("topic1"), "consumer1", "strategy1"); - JoinGroupRequestAndHeader(0.asInstanceOf[Short], 1, "", body) + def createUpdateMetadataRequest(versionId: Short): UpdateMetadataRequest = { + UpdateMetadataRequest( + versionId, + correlationId = 0, + clientId = "client1", + controllerId = 0, + controllerEpoch = 0, + partitionStateInfos = updateMetadataRequestPartitionStateInfo, + brokers.toSet + ) } - def createJoinGroupResponseAndHeader: JoinGroupResponseAndHeader = { - import scala.collection.JavaConversions._ - val body = new JoinGroupResponse(0.asInstanceOf[Short], 1, "consumer1", List(new TopicPartition("test11", 1))) - JoinGroupResponseAndHeader(1, body) + def createUpdateMetadataResponse: UpdateMetadataResponse = { + UpdateMetadataResponse( correlationId = 0, errorCode = 0) } } @@ -232,16 +270,17 @@ class RequestResponseSerializationTest extends JUnitSuite { private val topicMetadataResponse = SerializationTestUtils.createTestTopicMetadataResponse 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) - private val heartbeatRequest = SerializationTestUtils.createHeartbeatRequestAndHeader - private val heartbeatResponse = SerializationTestUtils.createHeartbeatResponseAndHeader - private val joinGroupRequest = SerializationTestUtils.createJoinGroupRequestAndHeader - private val joinGroupResponse = SerializationTestUtils.createJoinGroupResponseAndHeader + 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() { @@ -250,11 +289,12 @@ class RequestResponseSerializationTest extends JUnitSuite { collection.immutable.Seq(leaderAndIsrRequest, leaderAndIsrResponse, stopReplicaRequest, stopReplicaResponse, producerRequest, producerResponse, fetchRequest, offsetRequest, offsetResponse, topicMetadataRequest, - topicMetadataResponse, offsetCommitRequestV0, offsetCommitRequestV1, + topicMetadataResponse, + offsetCommitRequestV0, offsetCommitRequestV1, offsetCommitRequestV2, offsetCommitResponse, offsetFetchRequest, offsetFetchResponse, consumerMetadataRequest, consumerMetadataResponse, - consumerMetadataResponseNoCoordinator, heartbeatRequest, - heartbeatResponse, joinGroupRequest, joinGroupResponse) + updateMetadataRequestV0, updateMetadataRequestV1, updateMetdataResponse, + consumerMetadataResponseNoCoordinator) requestsAndResponses.foreach { original => val buffer = ByteBuffer.allocate(original.sizeInBytes) @@ -267,4 +307,39 @@ class RequestResponseSerializationTest extends JUnitSuite { assertEquals("The original and deserialized for " + original.getClass.getSimpleName + " should be the same.", original, deserialized) } } + + @Test + def testProduceResponseVersion() { + val oldClientResponse = ProducerResponse(1, Map( + TopicAndPartition("t1", 0) -> ProducerResponseStatus(0.toShort, 10001), + TopicAndPartition("t2", 0) -> ProducerResponseStatus(0.toShort, 20001) + )) + + val newClientResponse = ProducerResponse(1, Map( + TopicAndPartition("t1", 0) -> ProducerResponseStatus(0.toShort, 10001), + TopicAndPartition("t2", 0) -> ProducerResponseStatus(0.toShort, 20001) + ), 1, 100) + + // new response should have 4 bytes more than the old response since delayTime is an INT32 + assertEquals(oldClientResponse.sizeInBytes + 4, newClientResponse.sizeInBytes) + + val buffer = ByteBuffer.allocate(newClientResponse.sizeInBytes) + newClientResponse.writeTo(buffer) + buffer.rewind() + assertEquals(ProducerResponse.readFrom(buffer).throttleTime, 100) + } + + @Test + def testFetchResponseVersion() { + val oldClientResponse = FetchResponse(1, Map( + TopicAndPartition("t1", 0) -> new FetchResponsePartitionData(messages = new ByteBufferMessageSet(new Message("first message".getBytes))) + ), 0) + + val newClientResponse = FetchResponse(1, Map( + TopicAndPartition("t1", 0) -> new FetchResponsePartitionData(messages = new ByteBufferMessageSet(new Message("first message".getBytes))) + ), 1, 100) + + // new response should have 4 bytes more than the old response since delayTime is an INT32 + assertEquals(oldClientResponse.sizeInBytes + 4, newClientResponse.sizeInBytes) + } } 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..2d3a9c39333bf --- /dev/null +++ b/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.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.cluster + +import java.nio.ByteBuffer + +import kafka.utils.Logging +import org.apache.kafka.common.protocol.SecurityProtocol +import org.junit.Test + +import scala.collection.mutable + +class BrokerEndPointTest extends 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 7bff96caca1dc..a42836c6fdf79 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 org.junit.Assert._ import collection.mutable.ArrayBuffer import org.junit.Test -import kafka.common.InvalidConfigException import kafka.producer.ProducerConfig import kafka.consumer.ConsumerConfig diff --git a/core/src/test/scala/unit/kafka/common/TopicTest.scala b/core/src/test/scala/unit/kafka/common/TopicTest.scala index 0fb25880c24ad..0482bf1a6c6c7 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 org.junit.Assert._ import collection.mutable.ArrayBuffer -import kafka.common.{Topic, InvalidTopicException} import org.junit.Test class TopicTest { @@ -58,4 +57,45 @@ class TopicTest { } } } + + @Test + def testTopicHasCollisionChars() = { + val falseTopics = List("start", "end", "middle", "many") + val trueTopics = List( + ".start", "end.", "mid.dle", ".ma.ny.", + "_start", "end_", "mid_dle", "_ma_ny." + ) + + falseTopics.foreach( t => + assertFalse(Topic.hasCollisionChars(t)) + ) + + trueTopics.foreach( t => + assertTrue(Topic.hasCollisionChars(t)) + ) + } + + @Test + def testTopicHasCollision() = { + val periodFirstMiddleLastNone = List(".topic", "to.pic", "topic.", "topic") + val underscoreFirstMiddleLastNone = List("_topic", "to_pic", "topic_", "topic") + + // Self + periodFirstMiddleLastNone.foreach { t => + assertTrue(Topic.hasCollision(t, t)) + } + underscoreFirstMiddleLastNone.foreach { t => + assertTrue(Topic.hasCollision(t, t)) + } + + // Same Position + periodFirstMiddleLastNone.zip(underscoreFirstMiddleLastNone).foreach { case (t1, t2) => + assertTrue(Topic.hasCollision(t1, t2)) + } + + // Different Position + periodFirstMiddleLastNone.zip(underscoreFirstMiddleLastNone.reverse).foreach { case (t1, t2) => + assertFalse(Topic.hasCollision(t1, t2)) + } + } } diff --git a/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala b/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala new file mode 100644 index 0000000000000..d43778e985f9c --- /dev/null +++ b/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala @@ -0,0 +1,61 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 kafka.integration.KafkaServerTestHarness +import kafka.server.KafkaConfig +import kafka.utils.{TestUtils, ZkUtils} +import org.junit.Test + +class ZkNodeChangeNotificationListenerTest extends KafkaServerTestHarness { + + override def generateConfigs() = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnect))) + + @Test + def testProcessNotification() { + val notificationHandler = new NotificationHandler { + @volatile var notification: String = _ + @volatile var invocationCount: Integer = 0 + override def processNotification(notificationMessage: String): Unit = { + notification = notificationMessage + invocationCount += 1 + } + } + + val seqNodeRoot = "/root" + val seqNodePrefix = "prefix" + val seqNodePath = seqNodeRoot + "/" + seqNodePrefix + val notificationMessage1 = "message1" + val notificationMessage2 = "message2" + val changeExpirationMs = 100 + + val notificationListener = new ZkNodeChangeNotificationListener(zkClient, seqNodeRoot, seqNodePrefix, notificationHandler, changeExpirationMs) + notificationListener.init() + + ZkUtils.createSequentialPersistentPath(zkClient, seqNodePath, notificationMessage1) + + TestUtils.waitUntilTrue(() => notificationHandler.invocationCount == 1 && notificationHandler.notification == notificationMessage1, "failed to send/process notification message in the timeout period.") + + /*There is no easy way to test that purging. Even if we mock kafka time with MockTime, the purging compares kafka time with the time stored in zookeeper stat and the + embeded zookeeper server does not provide a way to mock time. so to test purging we will have to use SystemTime.sleep(changeExpirationMs + 1) issue a write and check + Assert.assertEquals(1, ZkUtils.getChildren(zkClient, seqNodeRoot).size) however even after that the assertion can fail as the second node it self can be deleted + depending on how threads get scheduled.*/ + + ZkUtils.createSequentialPersistentPath(zkClient, seqNodePath, notificationMessage2) + TestUtils.waitUntilTrue(() => notificationHandler.invocationCount == 2 && notificationHandler.notification == notificationMessage2, "failed to send/process notification message in the timeout period.") + } +} 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 151ba7ccd006f..ca63c80646bb8 --- a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala @@ -21,44 +21,42 @@ package kafka.consumer import java.util.concurrent._ import java.util.concurrent.atomic._ import scala.collection._ -import junit.framework.Assert._ +import org.junit.Assert._ import kafka.message._ import kafka.server._ import kafka.utils.TestUtils._ import kafka.utils._ -import org.junit.Test +import org.junit.{Before, Test} import kafka.serializer._ -import kafka.cluster.{Broker, Cluster} -import org.scalatest.junit.JUnit3Suite import kafka.integration.KafkaServerTestHarness -class ConsumerIteratorTest extends JUnit3Suite with KafkaServerTestHarness { +class ConsumerIteratorTest extends 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)) + @Before override def setUp() { - super.setUp + 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) } @@ -80,9 +78,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 index 9ceae222ca5bf..6c22e8bfeb104 100644 --- a/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala @@ -15,23 +15,22 @@ * limitations under the License. */ -package unit.kafka.consumer +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.consumer._ import kafka.utils.{TestUtils, Logging, ZkUtils, Json} -import junit.framework.Assert._ +import org.junit.Assert._ import kafka.common.TopicAndPartition -import unit.kafka.consumer.PartitionAssignorTest.StaticSubscriptionInfo -import kafka.consumer.ConsumerThreadId -import unit.kafka.consumer.PartitionAssignorTest.Scenario -import unit.kafka.consumer.PartitionAssignorTest.WildcardSubscriptionInfo +import kafka.consumer.PartitionAssignorTest.StaticSubscriptionInfo +import kafka.consumer.PartitionAssignorTest.Scenario +import kafka.consumer.PartitionAssignorTest.WildcardSubscriptionInfo +import org.junit.Test -class PartitionAssignorTest extends JUnit3Suite with Logging { +class PartitionAssignorTest extends Logging { + @Test def testRoundRobinPartitionAssignor() { val assignor = new RoundRobinAssignor @@ -55,6 +54,7 @@ class PartitionAssignorTest extends JUnit3Suite with Logging { }) } + @Test def testRangePartitionAssignor() { val assignor = new RangeAssignor (1 to PartitionAssignorTest.TestCaseCount).foreach (testCase => { @@ -87,7 +87,7 @@ private object PartitionAssignorTest extends Logging { private val MaxConsumerCount = 10 private val MaxStreamCount = 8 private val MaxTopicCount = 100 - private val MinTopicCount = 20 + private val MinTopicCount = 0 private val MaxPartitionCount = 120 private val MinPartitionCount = 8 @@ -127,7 +127,7 @@ private object PartitionAssignorTest extends Logging { "\n" + "Group : %s\n".format(group) + "Topic partition counts : %s\n".format(topicPartitionCounts) + - "Consumer subscriptions : %s\n".format(subscriptions) + "Consumer assignment : %s\n".format(subscriptions) } } @@ -164,7 +164,7 @@ private object PartitionAssignorTest extends Logging { verifyAssignmentIsUniform: Boolean = false) { val assignments = scenario.subscriptions.map{ case(consumer, subscription) => val ctx = new AssignmentContext("g1", consumer, excludeInternalTopics = true, zkClient) - assignor.assign(ctx) + assignor.assign(ctx).get(consumer) } // check for uniqueness (i.e., any partition should be assigned to exactly one consumer stream) @@ -187,10 +187,12 @@ private object PartitionAssignorTest extends Logging { // check for uniform assignment if (verifyAssignmentIsUniform) { val partitionCountForStream = partitionCountPerStream(globalAssignment) - 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) + 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) + } } } diff --git a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala index 4f124af5c3e94..2e18e92139c52 100644 --- a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala @@ -18,10 +18,11 @@ package kafka.consumer -import junit.framework.Assert._ +import org.junit.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.Test import kafka.server.OffsetManager +import kafka.coordinator.ConsumerCoordinator class TopicFilterTest extends JUnitSuite { @@ -37,8 +38,8 @@ class TopicFilterTest extends JUnitSuite { val topicFilter2 = new Whitelist(".+") assertTrue(topicFilter2.isTopicAllowed("alltopics", excludeInternalTopics = true)) - assertFalse(topicFilter2.isTopicAllowed(OffsetManager.OffsetsTopicName, excludeInternalTopics = true)) - assertTrue(topicFilter2.isTopicAllowed(OffsetManager.OffsetsTopicName, excludeInternalTopics = false)) + 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", excludeInternalTopics = true)) @@ -57,8 +58,8 @@ class TopicFilterTest extends JUnitSuite { assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = true)) assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = false)) - assertFalse(topicFilter1.isTopicAllowed(OffsetManager.OffsetsTopicName, excludeInternalTopics = true)) - assertTrue(topicFilter1.isTopicAllowed(OffsetManager.OffsetsTopicName, excludeInternalTopics = false)) + assertFalse(topicFilter1.isTopicAllowed(ConsumerCoordinator.OffsetsTopicName, excludeInternalTopics = true)) + assertTrue(topicFilter1.isTopicAllowed(ConsumerCoordinator.OffsetsTopicName, excludeInternalTopics = false)) } @Test diff --git a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala index e1d87112a2a58..cb595426318f4 100644 --- a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala @@ -17,35 +17,36 @@ package kafka.consumer -import junit.framework.Assert._ +import java.util.{Collections, Properties} + +import org.junit.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 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.common.MessageStreamsExistException +import kafka.utils._ +import org.I0Itec.zkclient.ZkClient +import org.apache.log4j.{Level, Logger} +import org.junit.{Test, After, Before} -class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHarness with Logging { +import scala.collection._ + +class ZookeeperConsumerConnectorTest extends 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" @@ -53,15 +54,18 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val consumer3 = "consumer3" val nMessages = 2 + @Before override def setUp() { super.setUp() dirs = new ZKGroupTopicDirs(group, topic) } + @After override def tearDown() { super.tearDown() } + @Test def testBasic() { val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) requestHandlerLogger.setLevel(Level.FATAL) @@ -78,7 +82,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 @@ -89,8 +93,8 @@ 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) @@ -104,7 +108,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar 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 @@ -114,7 +118,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)) { @@ -123,13 +127,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) 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 @@ -144,13 +148,13 @@ 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) 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 @@ -172,14 +176,14 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar requestHandlerLogger.setLevel(Level.ERROR) } - + @Test 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) waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) @@ -192,7 +196,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar 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 @@ -202,7 +206,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)) { @@ -211,13 +215,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) 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 @@ -232,13 +236,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) 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 @@ -252,10 +256,11 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar requestHandlerLogger.setLevel(Level.ERROR) } + @Test 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) TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1) @@ -263,7 +268,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar 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 @@ -275,13 +280,14 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar zkConsumerConnector1.shutdown } + @Test def testConsumerDecoder() { val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandler]) 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) TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1) @@ -314,14 +320,15 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar requestHandlerLogger.setLevel(Level.ERROR) } + @Test 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 createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 1, servers = servers) // send some messages to each broker - val sentMessages1 = sendMessages(configs.head, nMessages, "batch1", NoCompressionCodec, 1) + val sentMessages1 = sendMessages(servers, topic, nMessages) // create a consumer val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1)) @@ -339,85 +346,103 @@ 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("compression.codec", compression.codec.toString) - val producer: Producer[Int, String] = - createProducer(TestUtils.getBrokerListStrFromConfigs(configs), - encoder = classOf[StringEncoder].getName, - keyEncoder = classOf[IntEncoder].getName, - partitioner = classOf[FixedValuePartitioner].getName, - producerProps = 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 - } + @Test + 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("compression.codec", compression.codec.toString) - val producer: Producer[Int, String] = - createProducer(brokerList = TestUtils.getBrokerListStrFromConfigs(configs), - encoder = classOf[StringEncoder].getName, - keyEncoder = classOf[IntEncoder].getName, - partitioner = classOf[FixedValuePartitioner].getName, - producerProps = 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 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 + // 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 + + 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/controller/ControllerFailoverTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerFailoverTest.scala new file mode 100644 index 0000000000000..c93eca5162288 --- /dev/null +++ b/core/src/test/scala/unit/kafka/controller/ControllerFailoverTest.scala @@ -0,0 +1,181 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.controller + +import java.util.Properties +import java.util.concurrent.LinkedBlockingQueue + +import kafka.api.RequestOrResponse +import kafka.common.TopicAndPartition +import kafka.integration.KafkaServerTestHarness +import kafka.server.{KafkaConfig, KafkaServer} +import kafka.utils._ +import org.apache.kafka.common.metrics.Metrics +import org.apache.kafka.common.requests.{AbstractRequestResponse, AbstractRequest} +import org.apache.kafka.common.utils.SystemTime +import org.apache.log4j.{Level, Logger} +import org.junit.{After, Before, Test} + +import scala.collection.mutable + + +class ControllerFailoverTest extends KafkaServerTestHarness with Logging { + val log = Logger.getLogger(classOf[ControllerFailoverTest]) + val numNodes = 2 + val numParts = 1 + val msgQueueSize = 1 + val topic = "topic1" + val overridingProps = new Properties() + val metrics = new Metrics() + overridingProps.put(KafkaConfig.NumPartitionsProp, numParts.toString) + + override def generateConfigs() = TestUtils.createBrokerConfigs(numNodes, zkConnect) + .map(KafkaConfig.fromProps(_, overridingProps)) + + @Before + override def setUp() { + super.setUp() + } + + @After + override def tearDown() { + super.tearDown() + this.metrics.close() + } + + /** + * See @link{https://issues.apache.org/jira/browse/KAFKA-2300} + * for the background of this test case + */ + @Test + def testMetadataUpdate() { + log.setLevel(Level.INFO) + var controller: KafkaServer = this.servers.head; + // Find the current controller + val epochMap: mutable.Map[Int, Int] = mutable.Map.empty + for (server <- this.servers) { + epochMap += (server.config.brokerId -> server.kafkaController.epoch) + if(server.kafkaController.isActive()) { + controller = server + } + } + // Create topic with one partition + kafka.admin.AdminUtils.createTopic(controller.zkClient, topic, 1, 1) + val topicPartition = TopicAndPartition("topic1", 0) + var partitions = controller.kafkaController.partitionStateMachine.partitionsInState(OnlinePartition) + while (!partitions.contains(topicPartition)) { + partitions = controller.kafkaController.partitionStateMachine.partitionsInState(OnlinePartition) + Thread.sleep(100) + } + // Replace channel manager with our mock manager + controller.kafkaController.controllerContext.controllerChannelManager.shutdown() + val channelManager = new MockChannelManager(controller.kafkaController.controllerContext, + controller.kafkaController.config, metrics) + channelManager.startup() + controller.kafkaController.controllerContext.controllerChannelManager = channelManager + channelManager.shrinkBlockingQueue(0) + channelManager.stopSendThread(0) + // Spawn a new thread to block on the outgoing channel + // queue + val thread = new Thread(new Runnable { + def run() { + try { + controller.kafkaController.sendUpdateMetadataRequest(Seq(0), Set(topicPartition)) + log.info("Queue state %d %d".format(channelManager.queueCapacity(0), channelManager.queueSize(0))) + controller.kafkaController.sendUpdateMetadataRequest(Seq(0), Set(topicPartition)) + log.info("Queue state %d %d".format(channelManager.queueCapacity(0), channelManager.queueSize(0))) + } catch { + case e : Exception => { + log.info("Thread interrupted") + } + } + } + }) + thread.setName("mythread") + thread.start() + while (thread.getState() != Thread.State.WAITING) { + Thread.sleep(100) + } + // Assume that the thread is WAITING because it is + // blocked on the queue, so interrupt and move forward + thread.interrupt() + thread.join() + channelManager.resumeSendThread(0) + // Wait and find current controller + var found = false + var counter = 0 + while (!found && counter < 10) { + for (server <- this.servers) { + val previousEpoch = (epochMap get server.config.brokerId) match { + case Some(epoch) => + epoch + case None => + val msg = String.format("Missing element in epoch map %s", epochMap.mkString(", ")) + throw new IllegalStateException(msg) + } + + if (server.kafkaController.isActive + && (previousEpoch) < server.kafkaController.epoch) { + controller = server + found = true + } + } + if (!found) { + Thread.sleep(100) + counter += 1 + } + } + // Give it a shot to make sure that sending isn't blocking + try { + controller.kafkaController.sendUpdateMetadataRequest(Seq(0), Set(topicPartition)) + } catch { + case e : Throwable => { + fail(e) + } + } + } +} + +class MockChannelManager(private val controllerContext: ControllerContext, config: KafkaConfig, metrics: Metrics) + extends ControllerChannelManager(controllerContext, config, new SystemTime, metrics) { + + def stopSendThread(brokerId: Int) { + val requestThread = brokerStateInfo(brokerId).requestSendThread + requestThread.isRunning.set(false) + requestThread.interrupt + requestThread.join + } + + def shrinkBlockingQueue(brokerId: Int) { + val messageQueue = new LinkedBlockingQueue[QueueItem](1) + val brokerInfo = this.brokerStateInfo(brokerId) + this.brokerStateInfo.put(brokerId, brokerInfo.copy(messageQueue = messageQueue)) + } + + def resumeSendThread (brokerId: Int) { + this.startRequestSendThread(0) + } + + def queueCapacity(brokerId: Int): Int = { + this.brokerStateInfo(brokerId).messageQueue.remainingCapacity + } + + def queueSize(brokerId: Int): Int = { + this.brokerStateInfo(brokerId).messageQueue.size + } +} 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..3e763c3135e69 --- /dev/null +++ b/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala @@ -0,0 +1,378 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 org.junit.Assert._ +import kafka.common.{OffsetAndMetadata, TopicAndPartition} +import kafka.server.{OffsetManager, KafkaConfig} +import kafka.utils.TestUtils +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests.{OffsetCommitRequest, JoinGroupRequest} +import org.easymock.{IAnswer, 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 + type CommitOffsetCallbackParams = Map[TopicAndPartition, Short] + type CommitOffsetCallback = Map[TopicAndPartition, Short] => Unit + + val ConsumerMinSessionTimeout = 10 + val ConsumerMaxSessionTimeout = 200 + val DefaultSessionTimeout = 100 + 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 testCommitOffsetFromUnknownGroup() { + val groupId = "groupId" + val consumerId = "consumer" + val generationId = 1 + val tp = new TopicAndPartition("topic", 0) + val offset = OffsetAndMetadata(0) + + val commitOffsetResult = commitOffsets(groupId, consumerId, generationId, Map(tp -> offset), true) + assertEquals(Errors.ILLEGAL_GENERATION.code, commitOffsetResult(tp)) + } + + @Test + def testCommitOffsetWithDefaultGeneration() { + val groupId = "groupId" + val tp = new TopicAndPartition("topic", 0) + val offset = OffsetAndMetadata(0) + + val commitOffsetResult = commitOffsets(groupId, OffsetCommitRequest.DEFAULT_CONSUMER_ID, + OffsetCommitRequest.DEFAULT_GENERATION_ID, Map(tp -> offset), true) + assertEquals(Errors.NONE.code, commitOffsetResult(tp)) + } + + @Test + def testHeartbeatDuringRebalanceCausesRebalanceInProgress() { + 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, + DefaultSessionTimeout, 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 rebalance in progress + EasyMock.reset(offsetManager) + val heartbeatResult = heartbeat(groupId, assignedConsumerId, initialGenerationId, isCoordinatorForGroup = true) + assertEquals(Errors.REBALANCE_IN_PROGRESS.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 setupCommitOffsetsCallback: (Future[CommitOffsetCallbackParams], CommitOffsetCallback) = { + val responsePromise = Promise[CommitOffsetCallbackParams] + val responseFuture = responsePromise.future + val responseCallback: CommitOffsetCallback = offsets => responsePromise.success(offsets) + (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) + // should only have to wait as long as session timeout, but allow some extra time in case of an unexpected delay + Await.result(responseFuture, Duration(sessionTimeout+100, 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)) + } + + private def commitOffsets(groupId: String, + consumerId: String, + generationId: Int, + offsets: Map[TopicAndPartition, OffsetAndMetadata], + isCoordinatorForGroup: Boolean): CommitOffsetCallbackParams = { + val (responseFuture, responseCallback) = setupCommitOffsetsCallback + EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1) + EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) + val storeOffsetAnswer = new IAnswer[Unit] { + override def answer = responseCallback.apply(offsets.mapValues(_ => Errors.NONE.code)) + } + EasyMock.expect(offsetManager.storeOffsets(groupId, consumerId, generationId, offsets, responseCallback)) + .andAnswer(storeOffsetAnswer) + EasyMock.replay(offsetManager) + consumerCoordinator.handleCommitOffsets(groupId, consumerId, generationId, offsets, 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..5d812c258517c --- /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 org.junit.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..d8a79487ebbe5 --- /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 org.junit.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..79c691f5d8cb6 --- /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 org.junit.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 95303e098d40c..818673f698230 100644 --- a/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala +++ b/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala @@ -24,14 +24,13 @@ import kafka.utils.TestUtils import kafka.serializer._ import kafka.producer.{Producer, KeyedMessage} -import org.junit.Test +import org.junit.{After, Before, Test} import org.apache.log4j.{Level, Logger} -import org.scalatest.junit.JUnit3Suite -import junit.framework.Assert._ +import org.junit.Assert._ -class AutoOffsetResetTest extends JUnit3Suite with KafkaServerTestHarness with Logging { +class AutoOffsetResetTest extends KafkaServerTestHarness with Logging { - val configs = List(new KafkaConfig(TestUtils.createBrokerConfig(0))) + def generateConfigs() = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnect))) val topic = "test_topic" val group = "default_group" @@ -42,12 +41,14 @@ class AutoOffsetResetTest extends JUnit3Suite with KafkaServerTestHarness with L val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandler]) + @Before override def setUp() { super.setUp() // temporarily set request handler logger to a higher level requestHandlerLogger.setLevel(Level.FATAL) } + @After override def tearDown() { // restore set request handler logger to a higher level requestHandlerLogger.setLevel(Level.ERROR) @@ -78,7 +79,7 @@ class AutoOffsetResetTest extends JUnit3Suite with KafkaServerTestHarness with L TestUtils.createTopic(zkClient, topic, 1, 1, servers) val producer: Producer[String, Array[Byte]] = TestUtils.createProducer( - TestUtils.getBrokerListStrFromConfigs(configs), + TestUtils.getBrokerListStrFromServers(servers), keyEncoder = classOf[StringEncoder].getName) for(i <- 0 until numMessages) diff --git a/core/src/test/scala/unit/kafka/integration/BaseTopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/BaseTopicMetadataTest.scala new file mode 100644 index 0000000000000..20a4068228395 --- /dev/null +++ b/core/src/test/scala/unit/kafka/integration/BaseTopicMetadataTest.scala @@ -0,0 +1,295 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.io.File +import java.nio.ByteBuffer + +import kafka.admin.AdminUtils +import kafka.api.{TopicMetadataRequest, TopicMetadataResponse} +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.zk.ZooKeeperTestHarness +import org.apache.kafka.common.protocol.SecurityProtocol +import org.junit.Assert._ +import org.junit.{Test, After, Before} + +abstract class BaseTopicMetadataTest extends ZooKeeperTestHarness { + private var server1: KafkaServer = null + var brokerEndPoints: Seq[BrokerEndPoint] = null + var adHocConfigs: Seq[KafkaConfig] = null + val numConfigs: Int = 4 + + /* If this is `Some`, SSL will be enabled */ + protected def trustStoreFile: Option[File] + + @Before + override def setUp() { + super.setUp() + val props = createBrokerConfigs(numConfigs, zkConnect, enableSSL = trustStoreFile.isDefined, trustStoreFile = trustStoreFile) + 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)) + } + + @After + override def tearDown() { + server1.shutdown() + super.tearDown() + } + + @Test + def testTopicMetadataRequest { + // create topic + val topic = "test" + AdminUtils.createTopic(zkClient, topic, 1, 1) + + // create a topic metadata request + val topicMetadataRequest = new TopicMetadataRequest(List(topic), 0) + + val serializedMetadataRequest = ByteBuffer.allocate(topicMetadataRequest.sizeInBytes + 2) + topicMetadataRequest.writeTo(serializedMetadataRequest) + serializedMetadataRequest.rewind() + val deserializedMetadataRequest = TopicMetadataRequest.readFrom(serializedMetadataRequest) + + assertEquals(topicMetadataRequest, deserializedMetadataRequest) + } + + @Test + def testBasicTopicMetadata { + // create topic + val topic = "test" + createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 1, servers = Seq(server1)) + + val 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) + assertEquals("Expecting metadata only for 1 topic", 1, topicsMetadata.size) + assertEquals("Expecting metadata for the test topic", "test", topicsMetadata.head.topic) + val partitionMetadata = topicsMetadata.head.partitionsMetadata + assertEquals("Expecting metadata for 1 partition", 1, partitionMetadata.size) + assertEquals("Expecting partition id to be 0", 0, partitionMetadata.head.partitionId) + assertEquals(1, partitionMetadata.head.replicas.size) + } + + @Test + def testGetAllTopicMetadata { + // create topic + val topic1 = "testGetAllTopicMetadata1" + val topic2 = "testGetAllTopicMetadata2" + 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 + val topicsMetadata = ClientUtils.fetchTopicMetadata(Set.empty, brokerEndPoints, "TopicMetadataTest-testGetAllTopicMetadata", + 2000, 0).topicsMetadata + assertEquals(ErrorMapping.NoError, topicsMetadata.head.errorCode) + assertEquals(2, topicsMetadata.size) + assertEquals(ErrorMapping.NoError, topicsMetadata.head.partitionsMetadata.head.errorCode) + assertEquals(ErrorMapping.NoError, topicsMetadata.last.partitionsMetadata.head.errorCode) + val partitionMetadataTopic1 = topicsMetadata.head.partitionsMetadata + val partitionMetadataTopic2 = topicsMetadata.last.partitionsMetadata + assertEquals("Expecting metadata for 1 partition", 1, partitionMetadataTopic1.size) + assertEquals("Expecting partition id to be 0", 0, partitionMetadataTopic1.head.partitionId) + assertEquals(1, partitionMetadataTopic1.head.replicas.size) + assertEquals("Expecting metadata for 1 partition", 1, partitionMetadataTopic2.size) + assertEquals("Expecting partition id to be 0", 0, partitionMetadataTopic2.head.partitionId) + assertEquals(1, partitionMetadataTopic2.head.replicas.size) + } + + @Test + def testAutoCreateTopic { + // auto create topic + val topic = "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) + assertEquals("Expecting metadata for the test topic", topic, topicsMetadata.head.topic) + assertEquals(0, topicsMetadata.head.partitionsMetadata.size) + + // wait for leader to be elected + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) + TestUtils.waitUntilMetadataIsPropagated(Seq(server1), topic, 0) + + // retry the metadata for the auto created topic + 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) + val partitionMetadata = topicsMetadata.head.partitionsMetadata + assertEquals("Expecting metadata for 1 partition", 1, partitionMetadata.size) + assertEquals("Expecting partition id to be 0", 0, partitionMetadata.head.partitionId) + assertEquals(1, partitionMetadata.head.replicas.size) + assertTrue(partitionMetadata.head.leader.isDefined) + } + + @Test + def testAutoCreateTopicWithCollision { + // auto create topic + val topic1 = "testAutoCreate_Topic" + val topic2 = "testAutoCreate.Topic" + var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic1, topic2), brokerEndPoints, "TopicMetadataTest-testAutoCreateTopic", + 2000,0).topicsMetadata + assertEquals("Expecting metadata for 2 topics", 2, topicsMetadata.size) + assertEquals("Expecting metadata for topic1", topic1, topicsMetadata.head.topic) + assertEquals(ErrorMapping.LeaderNotAvailableCode, topicsMetadata.head.errorCode) + assertEquals("Expecting metadata for topic2", topic2, topicsMetadata(1).topic) + assertEquals("Expecting InvalidTopicCode for topic2 metadata", ErrorMapping.InvalidTopicCode, topicsMetadata(1).errorCode) + + // wait for leader to be elected + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 0) + TestUtils.waitUntilMetadataIsPropagated(Seq(server1), topic1, 0) + + // retry the metadata for the first auto created topic + topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic1), brokerEndPoints, "TopicMetadataTest-testBasicTopicMetadata", + 2000,0).topicsMetadata + assertEquals(ErrorMapping.NoError, topicsMetadata.head.errorCode) + assertEquals(ErrorMapping.NoError, topicsMetadata.head.partitionsMetadata.head.errorCode) + var partitionMetadata = topicsMetadata.head.partitionsMetadata + assertEquals("Expecting metadata for 1 partition", 1, partitionMetadata.size) + assertEquals("Expecting partition id to be 0", 0, partitionMetadata.head.partitionId) + assertEquals(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 + ""), 6000L) + }) + } + + @Test + 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")) + } + + + @Test + def testAliveBrokerListWithNoTopics { + checkMetadata(Seq(server1), 1) + } + + @Test + 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()) + } + + + @Test + 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/FetcherTest.scala b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala index 25845abbcad2e..c061597a6d0b4 100644 --- a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala +++ b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala @@ -19,81 +19,66 @@ package kafka.integration import java.util.concurrent._ import java.util.concurrent.atomic._ +import org.junit.{Test, After, Before} + import scala.collection._ -import junit.framework.Assert._ +import org.junit.Assert._ import kafka.cluster._ import kafka.server._ -import org.scalatest.junit.JUnit3Suite import kafka.consumer._ -import kafka.serializer._ -import kafka.producer.{KeyedMessage, Producer} -import kafka.utils.TestUtils._ import kafka.utils.TestUtils -class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { - +class FetcherTest extends 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 + @Before override def setUp() { super.setUp - createTopic(zkClient, topic, partitionReplicaAssignment = Map(0 -> Seq(configs.head.brokerId)), servers = servers) + 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) } + @After override def tearDown() { fetcher.stopConnections() super.tearDown } + @Test 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), - keyEncoder = classOf[StringEncoder].getName) - 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 3cf7c9bcd6449..bca0dccf049f8 --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -17,32 +17,80 @@ package kafka.integration +import java.util.Arrays + +import kafka.common.KafkaException import kafka.server._ -import kafka.utils.{Utils, TestUtils} -import org.scalatest.junit.JUnit3Suite +import kafka.utils.{CoreUtils, TestUtils} import kafka.zk.ZooKeeperTestHarness -import kafka.common.KafkaException +import org.junit.{After, Before} + +import scala.collection.mutable.Buffer /** * A test harness that brings up some number of broker nodes */ -trait KafkaServerTestHarness extends JUnit3Suite with ZooKeeperTestHarness { - - val configs: List[KafkaConfig] - var servers: List[KafkaServer] = null +trait KafkaServerTestHarness extends 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 + } + + def serverForId(id: Int) = servers.find(s => s.config.brokerId == id) + + def bootstrapUrl = servers.map(s => s.config.hostName + ":" + s.boundPort()).mkString(",") + + @Before override def setUp() { super.setUp if(configs.size <= 0) - throw new KafkaException("Must suply at least one server config.") - brokerList = TestUtils.getBrokerListStrFromConfigs(configs) - 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) } + @After 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/MinIsrConfigTest.scala b/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala new file mode 100644 index 0000000000000..397760123c712 --- /dev/null +++ b/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala @@ -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 kafka.integration + +import java.util.Properties + +import kafka.server.KafkaConfig +import kafka.utils.TestUtils +import org.junit.Test + +class MinIsrConfigTest extends KafkaServerTestHarness { + + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.MinInSyncReplicasProp, "5") + def generateConfigs() = TestUtils.createBrokerConfigs(1, zkConnect).map(KafkaConfig.fromProps(_, overridingProps)) + + @Test + def testDefaultKafkaConfig() { + assert(servers.head.getLogManager().defaultConfig.minInSyncReplicas == 5) + } + +} diff --git a/core/src/test/scala/unit/kafka/integration/PlaintextTopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/PlaintextTopicMetadataTest.scala new file mode 100644 index 0000000000000..176d251993205 --- /dev/null +++ b/core/src/test/scala/unit/kafka/integration/PlaintextTopicMetadataTest.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.integration + +class PlaintextTopicMetadataTest extends BaseTopicMetadataTest { + protected def trustStoreFile = None +} + 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 a5386a03b6295..e6f0c5431423a --- a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala +++ b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala @@ -18,32 +18,28 @@ package kafka.integration import java.nio.ByteBuffer -import junit.framework.Assert._ +import org.junit.Assert._ import kafka.api.{PartitionFetchInfo, FetchRequest, FetchRequestBuilder} import kafka.server.{KafkaRequestHandler, KafkaConfig} 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 org.junit.Test import scala.collection._ -import kafka.admin.AdminUtils import kafka.common.{TopicAndPartition, ErrorMapping, UnknownTopicOrPartitionException, OffsetOutOfRangeException} -import kafka.utils.{StaticPartitioner, 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 { +class PrimitiveApiTest extends ProducerConsumerTestHarness with ZooKeeperTestHarness { val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) - val port = TestUtils.choosePort() - val props = TestUtils.createBrokerConfig(0, port) - val config = new KafkaConfig(props) - val configs = List(config) + def generateConfigs() = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnect))) + @Test def testFetchRequestCanProperlySerialize() { val request = new FetchRequestBuilder() .clientId("test-client") @@ -60,6 +56,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with assertEquals(request, deserializedRequest) } + @Test def testEmptyFetchRequest() { val partitionRequests = immutable.Map[TopicAndPartition, PartitionFetchInfo]() val request = new FetchRequest(requestInfo = partitionRequests) @@ -67,6 +64,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with assertTrue(!fetched.hasError && fetched.data.size == 0) } + @Test def testDefaultEncoderProducerAndFetch() { val topic = "test-topic" @@ -87,16 +85,17 @@ 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")) } + @Test def testDefaultEncoderProducerAndFetchWithCompression() { val topic = "test-topic" val props = new Properties() props.put("compression.codec", "gzip") val stringProducer1 = TestUtils.createProducer[String, String]( - TestUtils.getBrokerListStrFromConfigs(configs), + TestUtils.getBrokerListStrFromServers(servers), encoder = classOf[StringEncoder].getName, keyEncoder = classOf[StringEncoder].getName, partitioner = classOf[StaticPartitioner].getName, @@ -109,11 +108,12 @@ 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")) } private def produceAndMultiFetch(producer: Producer[String, String]) { - createSimpleTopicsAndAwaitLeader(zkClient, List("test1", "test2", "test3", "test4")) + 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)); @@ -132,7 +132,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))) } } @@ -175,13 +175,14 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with requestHandlerLogger.setLevel(Level.ERROR) } + @Test def testProduceAndMultiFetch() { produceAndMultiFetch(producer) } private def multiProduce(producer: Producer[String, String]) { val topics = Map("test4" -> 0, "test1" -> 0, "test2" -> 0, "test3" -> 0) - createSimpleTopicsAndAwaitLeader(zkClient, topics.keys) + topics.keys.map(topic => TestUtils.createTopic(zkClient, topic, servers = servers)) val messages = new mutable.HashMap[String, Seq[String]] val builder = new FetchRequestBuilder() @@ -197,14 +198,16 @@ 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))) } } + @Test def testMultiProduce() { multiProduce(producer) } + @Test def testConsumerEmptyTopic() { val newTopic = "new-topic" TestUtils.createTopic(zkClient, newTopic, numPartitions = 1, replicationFactor = 1, servers = servers) @@ -213,14 +216,15 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with assertFalse(fetchResponse.messageSet(newTopic, 0).iterator.hasNext) } + @Test def testPipelinedProduceRequests() { val topics = Map("test4" -> 0, "test1" -> 0, "test2" -> 0, "test3" -> 0) - createSimpleTopicsAndAwaitLeader(zkClient, topics.keys) + 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] = TestUtils.createProducer[String, String]( - TestUtils.getBrokerListStrFromConfigs(configs), + TestUtils.getBrokerListStrFromServers(servers), encoder = classOf[StringEncoder].getName, keyEncoder = classOf[StringEncoder].getName, partitioner = classOf[StaticPartitioner].getName, @@ -262,18 +266,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))) - } - } - - /** - * 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. - */ - private def createSimpleTopicsAndAwaitLeader(zkClient: ZkClient, topics: Iterable[String]) { - for( topic <- topics ) { - AdminUtils.createTopic(zkClient, topic, partitions = 1, replicationFactor = 1) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, partition = 0) + 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 108c2e7f47ede..cc5954dfc5e55 100644 --- a/core/src/test/scala/unit/kafka/integration/ProducerConsumerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/ProducerConsumerTestHarness.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, @@ -18,29 +18,30 @@ package kafka.integration import kafka.consumer.SimpleConsumer -import org.scalatest.junit.JUnit3Suite +import org.junit.{After, Before} 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 +trait ProducerConsumerTestHarness extends KafkaServerTestHarness { + val host = "localhost" + var producer: Producer[String, String] = null + var consumer: SimpleConsumer = null + @Before override def setUp() { - super.setUp - producer = TestUtils.createProducer[String, String](TestUtils.getBrokerListStrFromConfigs(configs), - encoder = classOf[StringEncoder].getName, - keyEncoder = classOf[StringEncoder].getName, - partitioner = classOf[StaticPartitioner].getName) - consumer = new SimpleConsumer(host, port, 1000000, 64*1024, "") - } + super.setUp + 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() { - producer.close() - consumer.close() - super.tearDown - } + @After + override def tearDown() { + producer.close() + consumer.close() + super.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 eab4b5f619015..4d73be1e51c0c --- a/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala +++ b/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala @@ -17,52 +17,37 @@ package kafka.integration -import org.scalatest.junit.JUnit3Suite +import org.junit.{Test, After, Before} import kafka.zk.ZooKeeperTestHarness import kafka.utils.TestUtils._ -import junit.framework.Assert._ -import kafka.utils.{Utils, TestUtils} +import org.junit.Assert._ +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() - - // controlled.shutdown.enable is true by default - val configProps1 = TestUtils.createBrokerConfig(brokerId1, port1) - val configProps2 = TestUtils.createBrokerConfig(brokerId2, port2) - val configProps3 = TestUtils.createBrokerConfig(brokerId3, port3) - val configProps4 = TestUtils.createBrokerConfig(brokerId4, port4) - configProps4.put("controlled.shutdown.retry.backoff.ms", "100") - - var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] +class RollingBounceTest extends ZooKeeperTestHarness { val partitionId = 0 + var servers: Seq[KafkaServer] = null + @Before 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))) } + @After 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() } + @Test def testRollingBounce { // start all the brokers val topic1 = "new-topic1" diff --git a/core/src/test/scala/unit/kafka/integration/SslTopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/SslTopicMetadataTest.scala new file mode 100644 index 0000000000000..5ff9f35b27831 --- /dev/null +++ b/core/src/test/scala/unit/kafka/integration/SslTopicMetadataTest.scala @@ -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 kafka.integration + +import java.io.File + +class SslTopicMetadataTest extends BaseTopicMetadataTest { + protected def trustStoreFile = Some(File.createTempFile("truststore", ".jks")) +} diff --git a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala deleted file mode 100644 index 35dc071b1056e..0000000000000 --- a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala +++ /dev/null @@ -1,131 +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 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.utils.TestUtils -import kafka.utils.TestUtils._ -import kafka.server.{KafkaServer, KafkaConfig} -import kafka.api.TopicMetadataRequest -import kafka.common.ErrorMapping -import kafka.client.ClientUtils - -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)) - - override def setUp() { - super.setUp() - server1 = TestUtils.createServer(configs.head) - } - - override def tearDown() { - server1.shutdown() - super.tearDown() - } - - def testTopicMetadataRequest { - // create topic - val topic = "test" - AdminUtils.createTopic(zkClient, topic, 1, 1) - - // create a topic metadata request - val topicMetadataRequest = new TopicMetadataRequest(List(topic), 0) - - val serializedMetadataRequest = ByteBuffer.allocate(topicMetadataRequest.sizeInBytes + 2) - topicMetadataRequest.writeTo(serializedMetadataRequest) - serializedMetadataRequest.rewind() - val deserializedMetadataRequest = TopicMetadataRequest.readFrom(serializedMetadataRequest) - - assertEquals(topicMetadataRequest, deserializedMetadataRequest) - } - - def testBasicTopicMetadata { - // create topic - val topic = "test" - createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 1, servers = Seq(server1)) - - var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic),brokers,"TopicMetadataTest-testBasicTopicMetadata", - 2000,0).topicsMetadata - assertEquals(ErrorMapping.NoError, topicsMetadata.head.errorCode) - assertEquals(ErrorMapping.NoError, topicsMetadata.head.partitionsMetadata.head.errorCode) - assertEquals("Expecting metadata only for 1 topic", 1, topicsMetadata.size) - assertEquals("Expecting metadata for the test topic", "test", topicsMetadata.head.topic) - var partitionMetadata = topicsMetadata.head.partitionsMetadata - assertEquals("Expecting metadata for 1 partition", 1, partitionMetadata.size) - assertEquals("Expecting partition id to be 0", 0, partitionMetadata.head.partitionId) - assertEquals(1, partitionMetadata.head.replicas.size) - } - - def testGetAllTopicMetadata { - // create topic - val topic1 = "testGetAllTopicMetadata1" - val topic2 = "testGetAllTopicMetadata2" - 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", - 2000, 0).topicsMetadata - assertEquals(ErrorMapping.NoError, topicsMetadata.head.errorCode) - assertEquals(2, topicsMetadata.size) - assertEquals(ErrorMapping.NoError, topicsMetadata.head.partitionsMetadata.head.errorCode) - assertEquals(ErrorMapping.NoError, topicsMetadata.last.partitionsMetadata.head.errorCode) - val partitionMetadataTopic1 = topicsMetadata.head.partitionsMetadata - val partitionMetadataTopic2 = topicsMetadata.last.partitionsMetadata - assertEquals("Expecting metadata for 1 partition", 1, partitionMetadataTopic1.size) - assertEquals("Expecting partition id to be 0", 0, partitionMetadataTopic1.head.partitionId) - assertEquals(1, partitionMetadataTopic1.head.replicas.size) - assertEquals("Expecting metadata for 1 partition", 1, partitionMetadataTopic2.size) - assertEquals("Expecting partition id to be 0", 0, partitionMetadataTopic2.head.partitionId) - assertEquals(1, partitionMetadataTopic2.head.replicas.size) - } - - def testAutoCreateTopic { - // auto create topic - val topic = "testAutoCreateTopic" - var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic),brokers,"TopicMetadataTest-testAutoCreateTopic", - 2000,0).topicsMetadata - assertEquals(ErrorMapping.LeaderNotAvailableCode, topicsMetadata.head.errorCode) - assertEquals("Expecting metadata only for 1 topic", 1, topicsMetadata.size) - assertEquals("Expecting metadata for the test topic", topic, topicsMetadata.head.topic) - assertEquals(0, topicsMetadata.head.partitionsMetadata.size) - - // wait for leader to be elected - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) - TestUtils.waitUntilMetadataIsPropagated(Seq(server1), topic, 0) - - // retry the metadata for the auto created topic - topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic),brokers,"TopicMetadataTest-testBasicTopicMetadata", - 2000,0).topicsMetadata - assertEquals(ErrorMapping.NoError, topicsMetadata.head.errorCode) - assertEquals(ErrorMapping.NoError, topicsMetadata.head.partitionsMetadata.head.errorCode) - var partitionMetadata = topicsMetadata.head.partitionsMetadata - assertEquals("Expecting metadata for 1 partition", 1, partitionMetadata.size) - assertEquals("Expecting partition id to be 0", 0, partitionMetadata.head.partitionId) - assertEquals(1, partitionMetadata.head.replicas.size) - assertTrue(partitionMetadata.head.leader.isDefined) - } -} diff --git a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala old mode 100644 new mode 100755 index f44568cb25edf..e4f35762218c5 --- a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala @@ -17,40 +17,32 @@ package kafka.integration -import scala.collection.mutable.MutableList +import org.apache.kafka.common.config.ConfigException +import org.junit.{Test, After, Before} + 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.{DefaultEncoder, StringEncoder} +import kafka.consumer.{Consumer, ConsumerConfig} +import kafka.serializer.StringDecoder import kafka.server.{KafkaConfig, KafkaServer} -import kafka.utils.Utils +import kafka.utils.CoreUtils import kafka.utils.TestUtils._ import kafka.zk.ZooKeeperTestHarness +import org.junit.Assert._ -class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { +class UncleanLeaderElectionTest extends ZooKeeperTestHarness { val brokerId1 = 0 val brokerId2 = 1 - val port1 = choosePort() - val port2 = choosePort() - // 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 - val configProps1 = createBrokerConfig(brokerId1, port1) - val configProps2 = createBrokerConfig(brokerId2, port2) - 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)) - } + var configProps1: Properties = null + var configProps2: Properties = null var configs: Seq[KafkaConfig] = Seq.empty[KafkaConfig] var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] @@ -64,9 +56,19 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { val syncProducerLogger = Logger.getLogger(classOf[kafka.producer.SyncProducer]) val eventHandlerLogger = Logger.getLogger(classOf[kafka.producer.async.DefaultEventHandler[Object, Object]]) + @Before 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) @@ -74,9 +76,10 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { eventHandlerLogger.setLevel(Level.FATAL) } + @After override def tearDown() { - servers.map(server => shutdownServer(server)) - servers.map(server => Utils.rm(server.config.logDirs)) + servers.foreach(server => shutdownServer(server)) + servers.foreach(server => CoreUtils.rm(server.config.logDirs)) // restore log levels kafkaApisLogger.setLevel(Level.ERROR) @@ -89,13 +92,14 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { private def startBrokers(cluster: Seq[Properties]) { for (props <- cluster) { - val config = new KafkaConfig(props) + val config = KafkaConfig.fromProps(props) val server = createServer(config) configs ++= List(config) servers ++= List(server) } } + @Test def testUncleanLeaderElectionEnabled { // unclean leader election is enabled by default startBrokers(Seq(configProps1, configProps2)) @@ -106,10 +110,11 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { verifyUncleanLeaderElectionEnabled } + @Test def testUncleanLeaderElectionDisabled { - // disable unclean leader election - configProps1.put("unclean.leader.election.enable", String.valueOf(false)) - configProps2.put("unclean.leader.election.enable", String.valueOf(false)) + // 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 @@ -118,6 +123,7 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { verifyUncleanLeaderElectionDisabled } + @Test def testUncleanLeaderElectionEnabledByTopicOverride { // disable unclean leader election globally, but enable for our specific test topic configProps1.put("unclean.leader.election.enable", String.valueOf(false)) @@ -133,6 +139,7 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { verifyUncleanLeaderElectionEnabled } + @Test def testCleanLeaderElectionDisabledByTopicOverride { // enable unclean leader election globally, but disable for our specific test topic configProps1.put("unclean.leader.election.enable", String.valueOf(true)) @@ -148,6 +155,7 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { verifyUncleanLeaderElectionDisabled } + @Test def testUncleanLeaderElectionInvalidTopicOverride { startBrokers(Seq(configProps1)) @@ -155,7 +163,7 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { val topicProps = new Properties() topicProps.put("unclean.leader.election.enable", "invalid") - intercept[IllegalArgumentException] { + intercept[ConfigException] { AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1)), topicProps) } } @@ -172,14 +180,14 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { val followerId = if (leaderId == brokerId1) brokerId2 else brokerId1 debug("Follower for " + topic + " is: %s".format(followerId)) - produceMessage(topic, "first") + 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)) - produceMessage(topic, "second") + sendMessage(servers, topic, "second") assertEquals(List("first", "second"), consumeAllMessages(topic)) // shutdown leader and then restart follower @@ -189,7 +197,7 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { // wait until new leader is (uncleanly) elected waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, newLeaderOpt = Some(followerId)) - produceMessage(topic, "third") + sendMessage(servers, topic, "third") // second message was lost due to unclean election assertEquals(List("first", "third"), consumeAllMessages(topic)) @@ -207,14 +215,14 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { val followerId = if (leaderId == brokerId1) brokerId2 else brokerId1 debug("Follower for " + topic + " is: %s".format(followerId)) - produceMessage(topic, "first") + 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)) - produceMessage(topic, "second") + sendMessage(servers, topic, "second") assertEquals(List("first", "second"), consumeAllMessages(topic)) // shutdown leader and then restart follower @@ -226,7 +234,7 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { // message production and consumption should both fail while leader is down intercept[FailedToSendMessageException] { - produceMessage(topic, "third") + sendMessage(servers, topic, "third") } assertEquals(List.empty[String], consumeAllMessages(topic)) @@ -234,7 +242,7 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { servers.filter(server => server.config.brokerId == leaderId).map(server => server.startup()) waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, newLeaderOpt = Some(leaderId)) - produceMessage(topic, "third") + sendMessage(servers, topic, "third") waitUntilMetadataIsPropagated(servers, topic, partitionId) servers.filter(server => server.config.brokerId == leaderId).map(server => shutdownServer(server)) @@ -250,33 +258,16 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { server.awaitShutdown() } - private def produceMessage(topic: String, message: String) = { - val producer: Producer[String, Array[Byte]] = createProducer( - getBrokerListStrFromConfigs(configs), - keyEncoder = classOf[StringEncoder].getName) - producer.send(new KeyedMessage[String, Array[Byte]](topic, topic, message.getBytes)) - producer.close() - } - 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))(topic).head - - val messages = new MutableList[String] - val iter = messageStream.iterator - try { - while(iter.hasNext()) { - messages += new String(iter.next.message) // will throw a timeout exception if the message isn't there - } - } catch { - case e: ConsumerTimeoutException => - debug("consumer timed out after receiving " + messages.length + " message(s).") - } finally { - consumerConnector.shutdown - } - messages.toList + 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 d6248b09bb0f8..f4e012703211c 100644 --- a/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala @@ -17,8 +17,9 @@ package kafka.javaapi.consumer +import java.util.Properties + import kafka.server._ -import kafka.message._ import kafka.serializer._ import kafka.integration.KafkaServerTestHarness import kafka.producer.KeyedMessage @@ -28,30 +29,29 @@ import kafka.utils.{Logging, TestUtils} import kafka.consumer.{KafkaStream, ConsumerConfig} import kafka.zk.ZooKeeperTestHarness import kafka.common.MessageStreamsExistException +import org.junit.Test import scala.collection.JavaConversions -import org.scalatest.junit.JUnit3Suite import org.apache.log4j.{Level, Logger} -import junit.framework.Assert._ - +import org.junit.Assert._ -class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHarness with ZooKeeperTestHarness with Logging { - val zookeeperConnect = zkConnect +class ZookeeperConsumerConnectorTest extends 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 + @Test def testBasic() { val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) requestHandlerLogger.setLevel(Level.FATAL) @@ -60,10 +60,10 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar TestUtils.createTopic(zkClient, topic, numParts, 1, servers) // send some messages to each broker - val sentMessages1 = sendMessages(nMessages, "batch1") + 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()) @@ -82,32 +82,24 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar requestHandlerLogger.setLevel(Level.ERROR) } - def sendMessages(conf: KafkaConfig, + def sendMessages(servers: Seq[KafkaServer], messagesPerNode: Int, - header: String, - compressed: CompressionCodec): List[String] = { + header: String): List[String] = { var messages: List[String] = Nil - val producer: kafka.producer.Producer[Int, String] = - TestUtils.createProducer(TestUtils.getBrokerListStrFromConfigs(configs), - 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 + 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 } @@ -115,18 +107,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar 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 } diff --git a/core/src/test/scala/unit/kafka/javaapi/message/BaseMessageSetTestCases.scala b/core/src/test/scala/unit/kafka/javaapi/message/BaseMessageSetTestCases.scala index 726399e3c7a41..80f809ec59a26 100644 --- a/core/src/test/scala/unit/kafka/javaapi/message/BaseMessageSetTestCases.scala +++ b/core/src/test/scala/unit/kafka/javaapi/message/BaseMessageSetTestCases.scala @@ -17,7 +17,7 @@ package kafka.javaapi.message -import junit.framework.Assert._ +import org.junit.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.Test import kafka.utils.TestUtils diff --git a/core/src/test/scala/unit/kafka/javaapi/message/ByteBufferMessageSetTest.scala b/core/src/test/scala/unit/kafka/javaapi/message/ByteBufferMessageSetTest.scala index 383fcef02994f..fbdb000a1d8c4 100644 --- a/core/src/test/scala/unit/kafka/javaapi/message/ByteBufferMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/javaapi/message/ByteBufferMessageSetTest.scala @@ -17,7 +17,7 @@ package kafka.javaapi.message -import junit.framework.Assert._ +import org.junit.Assert._ import org.junit.Test import kafka.message.{DefaultCompressionCodec, CompressionCodec, NoCompressionCodec, Message} 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 d10e4f4ccbca5..9c4518c955ef5 --- a/core/src/test/scala/unit/kafka/log/CleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/CleanerTest.scala @@ -17,15 +17,20 @@ package kafka.log -import junit.framework.Assert._ -import org.scalatest.junit.JUnitSuite -import org.junit.{After, Test} -import java.nio._ import java.io.File -import scala.collection._ +import java.nio._ +import java.util.Properties +import java.util.concurrent.atomic.AtomicLong + import kafka.common._ -import kafka.utils._ import kafka.message._ +import kafka.utils._ +import org.apache.kafka.common.utils.Utils +import org.junit.Assert._ +import org.junit.{After, Test} +import org.scalatest.junit.JUnitSuite + +import scala.collection._ /** * Unit tests for the log cleaning logic @@ -33,13 +38,17 @@ import kafka.message._ class CleanerTest extends JUnitSuite { val dir = TestUtils.tempDir() - val logConfig = LogConfig(segmentSize=1024, maxIndexSize=1024, compact=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 +57,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 +82,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 +105,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 +158,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 +181,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 +225,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 +306,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 +434,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..95085f49144ab 100644 --- a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala @@ -17,9 +17,10 @@ package kafka.log +import java.io._ import java.nio._ import java.util.concurrent.atomic._ -import junit.framework.Assert._ +import org.junit.Assert._ import kafka.utils.TestUtils._ import kafka.message._ import org.junit.Test @@ -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 5bfa764638e92..70beb5f71204a --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -18,19 +18,27 @@ 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 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) { + val time = new MockTime() val segmentSize = 100 val deleteDelay = 1000 @@ -38,16 +46,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 +63,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,7 +111,7 @@ class LogCleanerIntegrationTest extends JUnitSuite { @After def teardown() { - Utils.rm(logDir) + CoreUtils.rm(logDir) } /* create a cleaner instance and logs with the given parameters */ @@ -100,8 +126,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, compact = true), + LogConfig(logProps), recoveryPoint = 0L, scheduler = time.scheduler, time = time) @@ -114,4 +145,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..51cd62c30ccc4 --- /dev/null +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.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.log + +import java.util.Properties + +import kafka.server.KafkaConfig +import kafka.server.KafkaServer +import kafka.utils.TestUtils +import org.apache.kafka.common.config.ConfigException +import org.junit.{Assert, Test} +import org.junit.Assert._ +import org.scalatest.Assertions._ + +class LogConfigTest { + + @Test + def testKafkaConfigToProps() { + val millisInHour = 60L * 60L * 1000L + val kafkaProps = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "") + kafkaProps.put(KafkaConfig.LogRollTimeHoursProp, "2") + kafkaProps.put(KafkaConfig.LogRollTimeJitterHoursProp, "2") + kafkaProps.put(KafkaConfig.LogRetentionTimeHoursProp, "2") + + val kafkaConfig = KafkaConfig.fromProps(kafkaProps) + val logProps = KafkaServer.copyKafkaConfigToLog(kafkaConfig) + assertEquals(2 * millisInHour, logProps.get(LogConfig.SegmentMsProp)) + assertEquals(2 * millisInHour, logProps.get(LogConfig.SegmentJitterMsProp)) + assertEquals(2 * millisInHour, logProps.get(LogConfig.RetentionMsProp)) + } + + @Test + def testFromPropsEmpty() { + val p = new Properties() + val config = LogConfig(p) + Assert.assertEquals(LogConfig(), config) + } + + @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 59bd8a981b3fb..816354f2b5927 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -18,38 +18,43 @@ package kafka.log import java.io._ -import junit.framework.Assert._ -import org.junit.Test -import org.scalatest.junit.JUnit3Suite -import kafka.server.{BrokerState, OffsetCheckpoint} +import java.util.Properties + import kafka.common._ +import kafka.server.OffsetCheckpoint import kafka.utils._ +import org.junit.Assert._ +import org.junit.{After, Before, Test} -class LogManagerTest extends JUnit3Suite { +class LogManagerTest { 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 - override def setUp() { - super.setUp() + @Before + def setUp() { logDir = TestUtils.tempDir() logManager = createLogManager() logManager.startup logDir = logManager.logDirs(0) } - override def tearDown() { + @After + def tearDown() { if(logManager != null) logManager.shutdown() - Utils.rm(logDir) - logManager.logDirs.map(Utils.rm(_)) - super.tearDown() + CoreUtils.rm(logDir) + logManager.logDirs.foreach(CoreUtils.rm(_)) } /** @@ -113,8 +118,11 @@ 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 = createLogManager() logManager.startup @@ -154,7 +162,10 @@ class LogManagerTest extends JUnit3Suite { @Test def testTimeBasedFlush() { logManager.shutdown() - val config = logConfig.copy(flushMs = 1000) + 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) diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala index 7b97e6a80753a..fa982b1c5fa10 100644 --- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala @@ -16,19 +16,15 @@ */ package kafka.log -import junit.framework.Assert._ +import org.junit.Assert._ import java.util.concurrent.atomic._ -import java.io.File -import java.io.RandomAccessFile -import java.util.Random import org.junit.{Test, After} -import org.scalatest.junit.JUnit3Suite import kafka.utils.TestUtils import kafka.message._ import kafka.utils.SystemTime import scala.collection._ -class LogSegmentTest extends JUnit3Suite { +class LogSegmentTest { val segments = mutable.ArrayBuffer[LogSegment]() @@ -39,7 +35,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 } @@ -226,5 +222,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 577d102fc2eb6..7f0d9d6ae51f4 --- 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.junit.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,9 +158,11 @@ 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) { @@ -137,17 +172,19 @@ class LogTest extends JUnitSuite { } 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) @@ -158,27 +195,29 @@ class LogTest extends JUnitSuite { 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).messageSet.head.offset) } - + /** * Test reading at the boundary of the log, specifically * - reading from the logEndOffset should give an empty message set @@ -187,7 +226,9 @@ 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) + 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) @@ -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(_)) @@ -226,34 +269,36 @@ class LogTest extends JUnitSuite { } 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).messageSet.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,18 +466,55 @@ 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) + 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 = 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).messageSet.head.offset) @@ -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,66 +676,72 @@ 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() { val log = new Log(logDir, @@ -585,11 +754,15 @@ class LogTest extends JUnitSuite { 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/log/OffsetIndexTest.scala b/core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala index 9213a5d9e95d0..dfd7b54fcf20c 100644 --- a/core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala +++ b/core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala @@ -18,7 +18,7 @@ package kafka.log import java.io._ -import junit.framework.Assert._ +import org.junit.Assert._ import java.util.{Collections, Arrays} import org.junit._ import org.scalatest.junit.JUnitSuite diff --git a/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala b/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala index 12ce39e665afd..f50daa43ef1d7 100644 --- a/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala +++ b/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala @@ -20,7 +20,7 @@ package kafka.log import java.nio._ import org.junit._ import org.scalatest.junit.JUnitSuite -import junit.framework.Assert._ +import org.junit.Assert._ class OffsetMapTest extends JUnitSuite { 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 4ea0489c9fd36..0000000000000 --- a/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala +++ /dev/null @@ -1,146 +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 kafka.consumer.SimpleConsumer -import kafka.server.{KafkaConfig, KafkaServer} -import kafka.utils.{TestUtils, Utils, Logging} -import kafka.api.FetchRequestBuilder -import kafka.producer.async.MissingConfigException -import kafka.serializer.Encoder -import kafka.zk.ZooKeeperTestHarness - -import java.util.Properties -import java.io.File - -import org.apache.log4j.spi.LoggingEvent -import org.apache.log4j.{PropertyConfigurator, Logger} -import org.junit.{After, Before, Test} -import org.scalatest.junit.JUnit3Suite - -import junit.framework.Assert._ - -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() { - // host missing - 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.logger.kafka.log4j", "INFO, KAFKA") - - try { - PropertyConfigurator.configure(props) - fail("Missing properties exception was expected !") - } catch { - case e: MissingConfigException => - } - - // topic missing - 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.logger.kafka.log4j", "INFO, KAFKA") - - try { - PropertyConfigurator.configure(props) - fail("Missing properties exception was expected !") - } catch { - case e: MissingConfigException => - } - } - - @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 = { - val 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.appender.KAFKA.RequiredNumAcks", "1") - props.put("log4j.appender.KAFKA.SyncSend", "true") - props.put("log4j.logger.kafka.log4j", "INFO, KAFKA") - 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 dd8847f5f709a..208994bbab964 100644 --- a/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala +++ b/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala @@ -18,7 +18,7 @@ package kafka.message import java.io.RandomAccessFile -import junit.framework.Assert._ +import org.junit.Assert._ import kafka.utils.TestUtils._ import kafka.log.FileMessageSet import org.scalatest.junit.JUnitSuite diff --git a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala index 4e45d965bc423..511060ebbb3a7 100644 --- a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala @@ -19,7 +19,7 @@ package kafka.message import java.nio._ import java.util.concurrent.atomic.AtomicLong -import junit.framework.Assert._ +import org.junit.Assert._ import org.junit.Test import kafka.utils.TestUtils @@ -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 6f0addcea64f1..f45bead84904b 100644 --- a/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala @@ -21,7 +21,7 @@ import java.io.ByteArrayOutputStream import scala.collection._ import org.scalatest.junit.JUnitSuite import org.junit._ -import junit.framework.Assert._ +import org.junit.Assert._ class MessageCompressionTest extends JUnitSuite { @@ -32,8 +32,6 @@ class MessageCompressionTest extends JUnitSuite { codecs += SnappyCompressionCodec if(isLZ4Available) codecs += LZ4CompressionCodec - if (izLZ4HCAvailable) - codecs += LZ4HCCompressionCodec for(codec <- codecs) testSimpleCompressDecompress(codec) } @@ -46,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 { @@ -74,14 +63,4 @@ class MessageCompressionTest extends JUnitSuite { case e: UnsatisfiedLinkError => false } } - - def izLZ4HCAvailable(): Boolean = { - try { - val lz4hc = new net.jpountz.lz4.LZ4BlockOutputStream(new ByteArrayOutputStream(), 1 << 16, - net.jpountz.lz4.LZ4Factory.fastestInstance().highCompressor()) - 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..3c12d13f5e57e --- a/core/src/test/scala/unit/kafka/message/MessageTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageTest.scala @@ -20,11 +20,12 @@ package kafka.message import java.nio._ import java.util.HashMap import scala.collection._ -import junit.framework.Assert._ +import org.junit.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..3993fdb1cc68b --- /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 org.junit.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/KafkaTimerTest.scala b/core/src/test/scala/unit/kafka/metrics/KafkaTimerTest.scala index 7df74050f9956..3b3e4c39e491f 100644 --- a/core/src/test/scala/unit/kafka/metrics/KafkaTimerTest.scala +++ b/core/src/test/scala/unit/kafka/metrics/KafkaTimerTest.scala @@ -18,12 +18,11 @@ package kafka.metrics */ import org.junit.Test -import org.scalatest.junit.JUnit3Suite import java.util.concurrent.TimeUnit -import junit.framework.Assert._ +import org.junit.Assert._ import com.yammer.metrics.core.{MetricsRegistry, Clock} -class KafkaTimerTest extends JUnit3Suite { +class KafkaTimerTest { @Test def testKafkaTimer() { 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..6ceb17b616950 --- /dev/null +++ b/core/src/test/scala/unit/kafka/metrics/MetricsTest.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.consumer + +import java.util.Properties + +import com.yammer.metrics.Metrics +import com.yammer.metrics.core.MetricPredicate +import org.junit.{After, Test} +import org.junit.Assert._ +import kafka.integration.KafkaServerTestHarness +import kafka.server._ +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 + +class MetricsTest extends 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 + + @After + 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 5f4d85254c384..6f07a7a2ef93c 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -17,32 +17,43 @@ package kafka.network; + import java.net._ +import javax.net.ssl._ import java.io._ + +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.security.auth.KafkaPrincipal +import org.apache.kafka.common.utils.SystemTime +import org.junit.Assert._ import org.junit._ import org.scalatest.junit.JUnitSuite import java.util.Random -import junit.framework.Assert._ import kafka.producer.SyncProducerConfig import kafka.api.ProducerRequest import java.nio.ByteBuffer import kafka.common.TopicAndPartition import kafka.message.ByteBufferMessageSet -import java.nio.channels.SelectionKey +import kafka.server.KafkaConfig import kafka.utils.TestUtils -class SocketServerTest extends JUnitSuite { +import scala.collection.Map - val server: SocketServer = new SocketServer(0, - host = null, - port = kafka.utils.TestUtils.choosePort, - numProcessorThreads = 1, - maxQueuedRequests = 50, - sendBufferSize = 300000, - recvBufferSize = 300000, - maxRequestSize = 50, - maxConnectionsPerIp = 5, - connectionsMaxIdleMs = 60*1000) +class SocketServerTest extends JUnitSuite { + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 0) + props.put("listeners", "PLAINTEXT://localhost:0,TRACE://localhost:0") + props.put("num.network.threads", "1") + props.put("socket.send.buffer.bytes", "300000") + props.put("socket.receive.buffer.bytes", "300000") + props.put("queued.max.requests", "50") + props.put("socket.request.max.bytes", "50") + props.put("max.connections.per.ip", "5") + props.put("connections.max.idle.ms", "60000") + val config = KafkaConfig.fromProps(props) + val metrics = new Metrics + val server = new SocketServer(config, metrics, new SystemTime) server.startup() def sendRequest(socket: Socket, id: Short, request: Array[Byte]) { @@ -67,20 +78,20 @@ 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() { + metrics.close() server.shutdown() } - - @Test - def simpleRequest() { - val socket = connect() + + private def producerRequestBytes: Array[Byte] = { val correlationId = -1 val clientId = SyncProducerConfig.DefaultClientId val ackTimeoutMs = SyncProducerConfig.DefaultAckTimeoutMs @@ -93,67 +104,166 @@ class SocketServerTest extends JUnitSuite { byteBuffer.rewind() val serializedBytes = new Array[Byte](byteBuffer.remaining) byteBuffer.get(serializedBytes) + serializedBytes + } + + @Test + def simpleRequest() { + val plainSocket = connect(protocol = SecurityProtocol.PLAINTEXT) + val traceSocket = connect(protocol = SecurityProtocol.TRACE) + val serializedBytes = producerRequestBytes - sendRequest(socket, 0, serializedBytes) + // Test PLAINTEXT socket + sendRequest(plainSocket, 0, serializedBytes) processRequest(server.requestChannel) - assertEquals(serializedBytes.toSeq, receiveResponse(socket).toSeq) + assertEquals(serializedBytes.toSeq, receiveResponse(plainSocket).toSeq) + + // Test TRACE socket + sendRequest(traceSocket, 0, serializedBytes) + processRequest(server.requestChannel) + assertEquals(serializedBytes.toSeq, receiveResponse(traceSocket).toSeq) } - @Test(expected = classOf[IOException]) + @Test def tooBigRequestIsRejected() { - val tooManyBytes = new Array[Byte](server.maxRequestSize + 1) + val tooManyBytes = new Array[Byte](server.config.socketRequestMaxBytes + 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 testNullResponse() { - val socket = connect() - val bytes = new Array[Byte](40) - sendRequest(socket, 0, bytes) - - val request = server.requestChannel.receiveRequest - // Since the response is not sent yet, the selection key should not be readable. - TestUtils.waitUntilTrue( - () => { (request.requestKey.asInstanceOf[SelectionKey].interestOps & SelectionKey.OP_READ) != SelectionKey.OP_READ }, - "Socket key shouldn't be available for read") - - server.requestChannel.sendResponse(new RequestChannel.Response(0, request, null)) - - // 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. - TestUtils.waitUntilTrue( - () => { (request.requestKey.asInstanceOf[SelectionKey].interestOps & SelectionKey.OP_READ) == SelectionKey.OP_READ }, - "Socket key should be available for reads") - } - - @Test(expected = classOf[IOException]) def testSocketsCloseOnShutdown() { // open a connection - val socket = connect() + 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(socket, 0, bytes) + sendRequest(plainSocket, 0, bytes) + sendRequest(traceSocket, 0, bytes) processRequest(server.requestChannel) + + // make sure the sockets are open + server.acceptors.values.map(acceptor => assertFalse(acceptor.serverChannel.socket.isClosed)) // then shutdown the server server.shutdown() + + val largeChunkOfBytes = new Array[Byte](1000000) // doing a subsequent send should throw an exception as the connection should be closed. - sendRequest(socket, 0, bytes) + // 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 + } } - + @Test def testMaxConnectionsPerIp() { // make the maximum allowable number of connections and then leak them - val conns = (0 until server.maxConnectionsPerIp).map(i => connect()) - + val conns = (0 until server.config.maxConnectionsPerIp).map(_ => connect()) // now try one more (should fail) + val conn = connect() + conn.setSoTimeout(3000) + assertEquals(-1, conn.getInputStream().read()) + conn.close() + + // it should succeed after closing one connection + val address = conns.head.getInetAddress + conns.head.close() + TestUtils.waitUntilTrue(() => server.connectionCount(address) < conns.length, + "Failed to decrement connection count after close") + val conn2 = connect() + val serializedBytes = producerRequestBytes + sendRequest(conn2, 0, serializedBytes) + val request = server.requestChannel.receiveRequest(2000) + assertNotNull(request) + conn2.close() + conns.tail.foreach(_.close()) + } + + @Test + def testMaxConnectionsPerIPOverrides() { + val overrideNum = 6 + val overrides = Map("localhost" -> overrideNum) + val overrideProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 0) + val serverMetrics = new Metrics() + val overrideServer: SocketServer = new SocketServer(KafkaConfig.fromProps(overrideProps), serverMetrics, new SystemTime()) try { - val conn = connect() - sendRequest(conn, 100, "hello".getBytes) - assertEquals(-1, conn.getInputStream().read()) - } catch { - case e: IOException => // this is good + 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()) + conn.close() + conns.foreach(_.close()) + } finally { + overrideServer.shutdown() + serverMetrics.close() + } + } + + @Test + def testSSLSocketServer(): Unit = { + val trustStoreFile = File.createTempFile("truststore", ".jks") + val overrideProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 0, enableSSL = true, trustStoreFile = Some(trustStoreFile)) + overrideProps.put("listeners", "SSL://localhost:0") + + val serverMetrics = new Metrics + val overrideServer: SocketServer = new SocketServer(KafkaConfig.fromProps(overrideProps), serverMetrics, new SystemTime) + overrideServer.startup() + try { + val sslContext = SSLContext.getInstance("TLSv1.2") + sslContext.init(null, Array(TestUtils.trustAllCerts), new java.security.SecureRandom()) + val socketFactory = sslContext.getSocketFactory + val sslSocket = socketFactory.createSocket("localhost", overrideServer.boundPort(SecurityProtocol.SSL)).asInstanceOf[SSLSocket] + sslSocket.setNeedClientAuth(false) + + val correlationId = -1 + val clientId = SyncProducerConfig.DefaultClientId + val ackTimeoutMs = SyncProducerConfig.DefaultAckTimeoutMs + val ack = SyncProducerConfig.DefaultRequiredAcks + 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) + + sendRequest(sslSocket, 0, serializedBytes) + processRequest(overrideServer.requestChannel) + assertEquals(serializedBytes.toSeq, receiveResponse(sslSocket).toSeq) + sslSocket.close() + } finally { + overrideServer.shutdown() + serverMetrics.close() } } + + @Test + def testSessionPrincipal(): Unit = { + val socket = connect() + val bytes = new Array[Byte](40) + sendRequest(socket, 0, bytes) + assertEquals(KafkaPrincipal.ANONYMOUS, server.requestChannel.receiveRequest().session.principal) + socket.close() + } + } 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 1db6ac329f7b5..b54f30e1bbd5a --- a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala @@ -19,33 +19,26 @@ package kafka.producer import java.util.Properties import java.util.concurrent.LinkedBlockingQueue -import junit.framework.Assert._ +import org.junit.Assert._ import org.easymock.EasyMock import org.junit.Test import kafka.api._ -import kafka.cluster.Broker +import kafka.cluster.BrokerEndPoint import kafka.common._ import kafka.message._ import kafka.producer.async._ import kafka.serializer._ import kafka.server.KafkaConfig import kafka.utils.TestUtils._ -import org.scalatest.junit.JUnit3Suite import scala.collection.Map import scala.collection.mutable.ArrayBuffer import kafka.utils._ -class AsyncProducerTest extends JUnit3Suite { - val props = createBrokerConfigs(1) - val configs = props.map(p => new KafkaConfig(p)) - - override def setUp() { - super.setUp() - } - - override def tearDown() { - super.tearDown() - } +class AsyncProducerTest { + // 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(",") @Test def testProducerQueueSize() { @@ -61,7 +54,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") @@ -86,7 +79,7 @@ class AsyncProducerTest extends JUnit3Suite { def testProduceAfterClosed() { val produceData = getProduceData(10) val producer = createProducer[String, String]( - getBrokerListStrFromConfigs(configs), + brokerList, encoder = classOf[StringEncoder].getName) producer.close @@ -162,9 +155,9 @@ 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) + 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)) @@ -212,7 +205,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) @@ -229,11 +222,11 @@ 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, Utils.readString(d.message.payload))) + 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) @@ -244,7 +237,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 @@ -274,7 +267,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 @@ -308,7 +301,7 @@ class AsyncProducerTest extends JUnit3Suite { // no need to retry since the send will always fail props.put("message.send.max.retries", "0") val producer= createProducer[String, String]( - brokerList = getBrokerListStrFromConfigs(configs), + brokerList = brokerList, encoder = classOf[DefaultEncoder].getName, keyEncoder = classOf[DefaultEncoder].getName, producerProps = props) @@ -326,7 +319,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 @@ -364,7 +357,7 @@ class AsyncProducerTest extends JUnit3Suite { @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) @@ -467,7 +460,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 dd71d81041e1b..dd96d298d7cfd --- a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala @@ -17,30 +17,26 @@ package kafka.producer -import org.scalatest.TestFailedException -import org.scalatest.junit.JUnit3Suite -import kafka.consumer.SimpleConsumer -import kafka.message.Message -import kafka.server.{KafkaConfig, KafkaRequestHandler, KafkaServer} -import kafka.zk.ZooKeeperTestHarness -import org.apache.log4j.{Level, Logger} -import org.junit.Test -import kafka.utils._ import java.util +import java.util.Properties + import kafka.admin.AdminUtils -import util.Properties import kafka.api.FetchRequestBuilder -import org.junit.Assert.assertTrue -import org.junit.Assert.assertFalse -import org.junit.Assert.assertEquals import kafka.common.{ErrorMapping, FailedToSendMessageException} +import kafka.consumer.SimpleConsumer +import kafka.message.Message import kafka.serializer.StringEncoder +import kafka.server.{KafkaConfig, KafkaRequestHandler, KafkaServer} +import kafka.utils._ +import kafka.zk.ZooKeeperTestHarness +import org.apache.log4j.{Level, Logger} +import org.junit.Assert._ +import org.junit.{After, Before, Test} +import org.scalatest.exceptions.TestFailedException -class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ +class ProducerTest extends 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,31 +44,43 @@ 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, false) - props1.put("num.partitions", "4") - private val config1 = new KafkaConfig(props1) - private val props2 = TestUtils.createBrokerConfig(brokerId2, port2, false) - 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 + } + + @Before 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) } + @After override def tearDown() { // restore set request handler logger to a higher level requestHandlerLogger.setLevel(Level.ERROR) @@ -84,8 +92,8 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ 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() } @@ -114,7 +122,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ } val producer2 = TestUtils.createProducer[String, String]( - brokerList = "localhost:80," + TestUtils.getBrokerListStrFromConfigs(Seq(config1)), + brokerList = "localhost:80," + TestUtils.getBrokerListStrFromServers(Seq(server1)), encoder = classOf[StringEncoder].getName, keyEncoder = classOf[StringEncoder].getName) @@ -127,7 +135,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ } val producer3 = TestUtils.createProducer[String, String]( - brokerList = TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)), + brokerList = TestUtils.getBrokerListStrFromServers(Seq(server1, server2)), encoder = classOf[StringEncoder].getName, keyEncoder = classOf[StringEncoder].getName) @@ -143,14 +151,14 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ @Test def testSendToNewTopic() { val props1 = new util.Properties() - props1.put("request.required.acks", "2") + props1.put("request.required.acks", "-1") val topic = "new-topic" // create topic with 1 partition and await leadership TestUtils.createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 2, servers = servers) val producer1 = TestUtils.createProducer[String, String]( - brokerList = TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)), + brokerList = TestUtils.getBrokerListStrFromServers(Seq(server1, server2)), encoder = classOf[StringEncoder].getName, keyEncoder = classOf[StringEncoder].getName, partitioner = classOf[StaticPartitioner].getName, @@ -165,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) @@ -181,24 +189,20 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ // no need to retry since the send will always fail props2.put("message.send.max.retries", "0") - val producer2 = TestUtils.createProducer[String, String]( - brokerList = TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)), - encoder = classOf[StringEncoder].getName, - keyEncoder = classOf[StringEncoder].getName, - partitioner = classOf[StaticPartitioner].getName, - producerProps = props2) - 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 => - // this is expected + case iae: IllegalArgumentException => // this is expected case e: Throwable => fail("Not expected", e) - } - finally { - producer2.close() + } } @@ -217,7 +221,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ servers = servers) val producer = TestUtils.createProducer[String, String]( - brokerList = TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)), + brokerList = TestUtils.getBrokerListStrFromServers(Seq(server1, server2)), encoder = classOf[StringEncoder].getName, keyEncoder = classOf[StringEncoder].getName, partitioner = classOf[StaticPartitioner].getName, @@ -251,7 +255,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ 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) @@ -271,7 +275,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ props.put("message.send.max.retries", "0") props.put("client.id","ProducerTest-testAsyncSendCanCorrectlyFailWithTimeout") val producer = TestUtils.createProducer[String, String]( - brokerList = TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)), + brokerList = TestUtils.getBrokerListStrFromServers(Seq(server1, server2)), encoder = classOf[StringEncoder].getName, keyEncoder = classOf[StringEncoder].getName, partitioner = classOf[StaticPartitioner].getName, @@ -286,7 +290,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ // 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) @@ -318,7 +322,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ @Test def testSendNullMessage() { val producer = TestUtils.createProducer[String, String]( - brokerList = TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)), + brokerList = TestUtils.getBrokerListStrFromServers(Seq(server1, server2)), encoder = classOf[StringEncoder].getName, keyEncoder = classOf[StringEncoder].getName, partitioner = classOf[StaticPartitioner].getName) diff --git a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala index 24deea06753e5..90689f6935df5 100644 --- a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala @@ -18,27 +18,30 @@ package kafka.producer import java.net.SocketTimeoutException -import junit.framework.Assert +import java.util.Properties + +import org.junit.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); +class SyncProducerTest extends KafkaServerTestHarness { + private val messageBytes = new Array[Byte](2) // turning off controlled shutdown since testProducerCanTimeout() explicitly shuts down request handler pool. - val configs = List(new KafkaConfig(TestUtils.createBrokerConfigs(1, false).head)) - val zookeeperConnect = TestZKUtils.zookeeperConnect + 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 @@ -73,7 +76,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 @@ -90,7 +94,7 @@ 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)) TestUtils.createTopic(zkClient, "test", numPartitions = 1, replicationFactor = 1, servers = servers) @@ -113,11 +117,12 @@ 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)) @@ -143,7 +148,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)) @@ -189,7 +194,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 +220,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 +232,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/security/auth/AclTest.scala b/core/src/test/scala/unit/kafka/security/auth/AclTest.scala new file mode 100644 index 0000000000000..d4de1795256ca --- /dev/null +++ b/core/src/test/scala/unit/kafka/security/auth/AclTest.scala @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package unit.kafka.security.auth + +import kafka.security.auth._ +import kafka.utils.Json +import org.apache.kafka.common.security.auth.KafkaPrincipal +import org.junit.{Test, Assert} +import org.scalatest.junit.JUnitSuite + +class AclTest extends JUnitSuite { + + val AclJson = "{\"version\": 1, \"acls\": [{\"host\": \"host1\",\"permissionType\": \"Deny\",\"operation\": \"READ\", \"principal\": \"User:alice\" }, " + + "{ \"host\": \"*\" , \"permissionType\": \"Allow\", \"operation\": \"Read\", \"principal\": \"User:bob\" }, " + + "{ \"host\": \"host1\", \"permissionType\": \"Deny\", \"operation\": \"Read\" , \"principal\": \"User:bob\"} ]}" + + @Test + def testAclJsonConversion(): Unit = { + val acl1 = new Acl(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "alice"), Deny, "host1" , Read) + val acl2 = new Acl(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "bob"), Allow, "*", Read) + val acl3 = new Acl(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "bob"), Deny, "host1", Read) + + val acls = Set[Acl](acl1, acl2, acl3) + val jsonAcls = Json.encode(Acl.toJsonCompatibleMap(acls)) + + Assert.assertEquals(acls, Acl.fromJson(jsonAcls)) + Assert.assertEquals(acls, Acl.fromJson(AclJson)) + } + +} diff --git a/core/src/test/scala/unit/kafka/security/auth/OperationTest.scala b/core/src/test/scala/unit/kafka/security/auth/OperationTest.scala new file mode 100644 index 0000000000000..2f15f9f22e574 --- /dev/null +++ b/core/src/test/scala/unit/kafka/security/auth/OperationTest.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 unit.kafka.security.auth + +import kafka.common.{KafkaException} +import kafka.security.auth.{Operation, Read} +import org.junit.{Test, Assert} +import org.scalatest.junit.JUnitSuite + +class OperationTest extends JUnitSuite { + + @Test + def testFromString(): Unit = { + val op = Operation.fromString("READ") + Assert.assertEquals(Read, op) + + try { + Operation.fromString("badName") + fail("Expected exception on invalid operation name.") + } catch { + case e: KafkaException => "Expected." + } + } + +} diff --git a/core/src/test/scala/unit/kafka/security/auth/PermissionTypeTest.scala b/core/src/test/scala/unit/kafka/security/auth/PermissionTypeTest.scala new file mode 100644 index 0000000000000..05189859eac7b --- /dev/null +++ b/core/src/test/scala/unit/kafka/security/auth/PermissionTypeTest.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 unit.kafka.security.auth + +import kafka.common.KafkaException +import kafka.security.auth.{Allow, PermissionType} +import org.junit.{Test, Assert} +import org.scalatest.junit.JUnitSuite + +class PermissionTypeTest extends JUnitSuite { + + @Test + def testFromString(): Unit = { + val permissionType = PermissionType.fromString("Allow") + Assert.assertEquals(Allow, permissionType) + + try { + PermissionType.fromString("badName") + fail("Expected exception on invalid PermissionType name.") + } catch { + case e: KafkaException => "Expected." + } + } + +} diff --git a/core/src/test/scala/unit/kafka/security/auth/ResourceTypeTest.scala b/core/src/test/scala/unit/kafka/security/auth/ResourceTypeTest.scala new file mode 100644 index 0000000000000..a632d372941b9 --- /dev/null +++ b/core/src/test/scala/unit/kafka/security/auth/ResourceTypeTest.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 unit.kafka.security.auth + +import kafka.common.KafkaException +import kafka.security.auth.{ResourceType, Topic} +import org.junit.{Test, Assert} +import org.scalatest.junit.JUnitSuite + +class ResourceTypeTest extends JUnitSuite { + + @Test + def testFromString(): Unit = { + val resourceType = ResourceType.fromString("Topic") + Assert.assertEquals(Topic, resourceType) + + try { + ResourceType.fromString("badName") + fail("Expected exception on invalid ResourceType name.") + } catch { + case e: KafkaException => "Expected." + } + } + +} diff --git a/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala b/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala new file mode 100644 index 0000000000000..655bc2040d8d2 --- /dev/null +++ b/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.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.security.auth + +import java.util.UUID + +import kafka.network.RequestChannel.Session +import kafka.security.auth.Acl.WildCardHost +import kafka.server.KafkaConfig +import kafka.utils.{ZkUtils, TestUtils} +import kafka.zk.ZooKeeperTestHarness +import org.apache.kafka.common.security.auth.KafkaPrincipal +import org.junit.Assert._ +import org.junit.{Before, Test} + +class SimpleAclAuthorizerTest extends ZooKeeperTestHarness { + + var simpleAclAuthorizer = new SimpleAclAuthorizer + val testPrincipal = Acl.WildCardPrincipal + val testHostName = "test.host.com" + var session = new Session(testPrincipal, testHostName) + var resource: Resource = null + val superUsers = "User:superuser1, User:superuser2" + val username = "alice" + var config: KafkaConfig = null + + @Before + override def setUp() { + super.setUp() + + val props = TestUtils.createBrokerConfig(0, zkConnect) + props.put(SimpleAclAuthorizer.SuperUsersProp, superUsers) + + config = KafkaConfig.fromProps(props) + simpleAclAuthorizer.configure(config.originals) + resource = new Resource(Topic, UUID.randomUUID().toString) + } + + @Test + def testTopicAcl() { + val user1 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username) + val user2 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "rob") + val user3 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "batman") + val host1 = "host1" + val host2 = "host2" + + //user1 has READ access from host1 and host2. + val acl1 = new Acl(user1, Allow, host1, Read) + val acl2 = new Acl(user1, Allow, host2, Read) + + //user1 does not have READ access from host1. + val acl3 = new Acl(user1, Deny, host1, Read) + + //user1 has Write access from host1 only. + val acl4 = new Acl(user1, Allow, host1, Write) + + //user1 has DESCRIBE access from all hosts. + val acl5 = new Acl(user1, Allow, WildCardHost, Describe) + + //user2 has READ access from all hosts. + val acl6 = new Acl(user2, Allow, WildCardHost, Read) + + //user3 has WRITE access from all hosts. + val acl7 = new Acl(user3, Allow, WildCardHost, Write) + + val acls = Set[Acl](acl1, acl2, acl3, acl4, acl5, acl6, acl7) + + changeAclAndVerify(Set.empty[Acl], acls, Set.empty[Acl]) + + val host1Session = new Session(user1, host1) + val host2Session = new Session(user1, host2) + + assertTrue("User1 should have READ access from host2", simpleAclAuthorizer.authorize(host2Session, Read, resource)) + assertFalse("User1 should not have READ access from host1 due to denyAcl", simpleAclAuthorizer.authorize(host1Session, Read, resource)) + assertTrue("User1 should have WRITE access from host1", simpleAclAuthorizer.authorize(host1Session, Write, resource)) + assertFalse("User1 should not have WRITE access from host2 as no allow acl is defined", simpleAclAuthorizer.authorize(host2Session, Write, resource)) + assertTrue("User1 should not have DESCRIBE access from host1", simpleAclAuthorizer.authorize(host1Session, Describe, resource)) + assertTrue("User1 should have DESCRIBE access from host2", simpleAclAuthorizer.authorize(host2Session, Describe, resource)) + assertFalse("User1 should not have edit access from host1", simpleAclAuthorizer.authorize(host1Session, Alter, resource)) + assertFalse("User1 should not have edit access from host2", simpleAclAuthorizer.authorize(host2Session, Alter, resource)) + + //test if user has READ and write access they also get describe access + val user2Session = new Session(user2, host1) + val user3Session = new Session(user3, host1) + assertTrue("User2 should have DESCRIBE access from host1", simpleAclAuthorizer.authorize(user2Session, Describe, resource)) + assertTrue("User3 should have DESCRIBE access from host2", simpleAclAuthorizer.authorize(user3Session, Describe, resource)) + assertTrue("User2 should have READ access from host1", simpleAclAuthorizer.authorize(user2Session, Read, resource)) + assertTrue("User3 should have WRITE access from host2", simpleAclAuthorizer.authorize(user3Session, Write, resource)) + } + + @Test + def testDenyTakesPrecedence() { + val user = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username) + val host = "random-host" + val session = new Session(user, host) + + val allowAll = Acl.AllowAllAcl + val denyAcl = new Acl(user, Deny, host, All) + val acls = Set[Acl](allowAll, denyAcl) + + changeAclAndVerify(Set.empty[Acl], acls, Set.empty[Acl]) + + assertFalse("deny should take precedence over allow.", simpleAclAuthorizer.authorize(session, Read, resource)) + } + + @Test + def testAllowAllAccess() { + val allowAllAcl = Acl.AllowAllAcl + + changeAclAndVerify(Set.empty[Acl], Set[Acl](allowAllAcl), Set.empty[Acl]) + + val session = new Session(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "random"), "random.host") + assertTrue("allow all acl should allow access to all.", simpleAclAuthorizer.authorize(session, Read, resource)) + } + + @Test + def testSuperUserHasAccess() { + val denyAllAcl = new Acl(Acl.WildCardPrincipal, Deny, WildCardHost, All) + + changeAclAndVerify(Set.empty[Acl], Set[Acl](denyAllAcl), Set.empty[Acl]) + + val session1 = new Session(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "superuser1"), "random.host") + val session2 = new Session(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "superuser2"), "random.host") + + assertTrue("superuser always has access, no matter what acls.", simpleAclAuthorizer.authorize(session1, Read, resource)) + assertTrue("superuser always has access, no matter what acls.", simpleAclAuthorizer.authorize(session2, Read, resource)) + } + + @Test + def testWildCardAcls(): Unit = { + assertFalse("when acls = [], authorizer should fail close.", simpleAclAuthorizer.authorize(session, Read, resource)) + + val user1 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username) + val host1 = "host1" + val readAcl = new Acl(user1, Allow, host1, Read) + val wildCardResource = new Resource(resource.resourceType, Resource.WildCardResource) + + val acls = changeAclAndVerify(Set.empty[Acl], Set[Acl](readAcl), Set.empty[Acl], wildCardResource) + + val host1Session = new Session(user1, host1) + assertTrue("User1 should have Read access from host1", simpleAclAuthorizer.authorize(host1Session, Read, resource)) + + //allow Write to specific topic. + val writeAcl = new Acl(user1, Allow, host1, Write) + changeAclAndVerify(Set.empty[Acl], Set[Acl](writeAcl), Set.empty[Acl]) + + //deny Write to wild card topic. + val denyWriteOnWildCardResourceAcl = new Acl(user1, Deny, host1, Write) + changeAclAndVerify(acls, Set[Acl](denyWriteOnWildCardResourceAcl), Set.empty[Acl], wildCardResource) + + assertFalse("User1 should not have Write access from host1", simpleAclAuthorizer.authorize(host1Session, Write, resource)) + } + + @Test + def testNoAclFound() { + assertFalse("when acls = [], authorizer should fail close.", simpleAclAuthorizer.authorize(session, Read, resource)) + } + + @Test + def testNoAclFoundOverride() { + val props = TestUtils.createBrokerConfig(1, zkConnect) + props.put(SimpleAclAuthorizer.AllowEveryoneIfNoAclIsFoundProp, "true") + + val cfg = KafkaConfig.fromProps(props) + val testAuthoizer: SimpleAclAuthorizer = new SimpleAclAuthorizer + testAuthoizer.configure(cfg.originals) + assertTrue("when acls = null or [], authorizer should fail open with allow.everyone = true.", testAuthoizer.authorize(session, Read, resource)) + } + + @Test + def testAclManagementAPIs() { + val user1 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username) + val user2 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "bob") + val host1 = "host1" + val host2 = "host2" + + val acl1 = new Acl(user1, Allow, host1, Read) + val acl2 = new Acl(user1, Allow, host1, Write) + val acl3 = new Acl(user2, Allow, host2, Read) + val acl4 = new Acl(user2, Allow, host2, Write) + + var acls = changeAclAndVerify(Set.empty[Acl], Set[Acl](acl1, acl2, acl3, acl4), Set.empty[Acl]) + + //test addAcl is additive + val acl5 = new Acl(user2, Allow, WildCardHost, Read) + acls = changeAclAndVerify(acls, Set[Acl](acl5), Set.empty[Acl]) + + //test get by principal name. + TestUtils.waitUntilTrue(() => Map(resource -> Set(acl1, acl2)) == simpleAclAuthorizer.getAcls(user1), "changes not propagated in timeout period") + TestUtils.waitUntilTrue(() => Map(resource -> Set(acl3, acl4, acl5)) == simpleAclAuthorizer.getAcls(user2), "changes not propagated in timeout period") + + val resourceToAcls = Map[Resource, Set[Acl]]( + new Resource(Topic, Resource.WildCardResource) -> Set[Acl](new Acl(user2, Allow, WildCardHost, Read)), + new Resource(Cluster, Resource.WildCardResource) -> Set[Acl](new Acl(user2, Allow, host1, Read)), + new Resource(ConsumerGroup, Resource.WildCardResource) -> acls, + new Resource(ConsumerGroup, "test-ConsumerGroup") -> acls + ) + + resourceToAcls foreach { case (key, value) => changeAclAndVerify(Set.empty[Acl], value, Set.empty[Acl], key) } + TestUtils.waitUntilTrue(() => resourceToAcls + (resource -> acls) == simpleAclAuthorizer.getAcls(), "changes not propagated in timeout period.") + + //test remove acl from existing acls. + acls = changeAclAndVerify(acls, Set.empty[Acl], Set(acl1, acl5)) + + //test remove all acls for resource + simpleAclAuthorizer.removeAcls(resource) + TestUtils.waitAndVerifyAcls(Set.empty[Acl], simpleAclAuthorizer, resource) + assertTrue(!ZkUtils.pathExists(zkClient, simpleAclAuthorizer.toResourcePath(resource))) + + //test removing last acl also deletes zookeeper path + acls = changeAclAndVerify(Set.empty[Acl], Set(acl1), Set.empty[Acl]) + changeAclAndVerify(acls, Set.empty[Acl], acls) + assertTrue(!ZkUtils.pathExists(zkClient, simpleAclAuthorizer.toResourcePath(resource))) + } + + @Test + def testLoadCache() { + val user1 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username) + val acl1 = new Acl(user1, Allow, "host-1", Read) + val acls = Set[Acl](acl1) + simpleAclAuthorizer.addAcls(acls, resource) + + val user2 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "bob") + val resource1 = new Resource(Topic, "test-2") + val acl2 = new Acl(user2, Deny, "host3", Read) + val acls1 = Set[Acl](acl2) + simpleAclAuthorizer.addAcls(acls1, resource1) + + ZkUtils.deletePathRecursive(zkClient, SimpleAclAuthorizer.AclChangedZkPath) + val authorizer = new SimpleAclAuthorizer + authorizer.configure(config.originals) + + assertEquals(acls, authorizer.getAcls(resource)) + assertEquals(acls1, authorizer.getAcls(resource1)) + } + + private def changeAclAndVerify(originalAcls: Set[Acl], addedAcls: Set[Acl], removedAcls: Set[Acl], resource: Resource = resource): Set[Acl] = { + var acls = originalAcls + + if(addedAcls.nonEmpty) { + simpleAclAuthorizer.addAcls(addedAcls, resource) + acls ++= addedAcls + } + + if(removedAcls.nonEmpty) { + simpleAclAuthorizer.removeAcls(removedAcls, resource) + acls --=removedAcls + } + + TestUtils.waitAndVerifyAcls(acls, simpleAclAuthorizer, resource) + + acls + } +} \ No newline at end of file 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..7b55f79b3bf2d --- a/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala @@ -17,36 +17,42 @@ package kafka.server -import org.scalatest.junit.JUnit3Suite +import org.junit.Assert._ +import kafka.utils.{TestUtils, CoreUtils, ZkUtils} import kafka.zk.ZooKeeperTestHarness -import junit.framework.Assert._ -import kafka.utils.{ZkUtils, Utils, TestUtils} +import org.apache.kafka.common.protocol.SecurityProtocol +import org.junit.{Test, After, Before} -class AdvertiseBrokerTest extends JUnit3Suite with ZooKeeperTestHarness { +class AdvertiseBrokerTest extends ZooKeeperTestHarness { var server : KafkaServer = null val brokerId = 0 val advertisedHostName = "routable-host" val advertisedPort = 1234 + @Before 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)) } + @After override def tearDown() { server.shutdown() - Utils.rm(server.config.logDirs) + CoreUtils.rm(server.config.logDirs) super.tearDown() } - + + @Test 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/ReplicaFetchTest.scala b/core/src/test/scala/unit/kafka/server/BaseReplicaFetchTest.scala similarity index 79% rename from core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala rename to core/src/test/scala/unit/kafka/server/BaseReplicaFetchTest.scala index da4bafc1e2a94..b744b94cf4e5a 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/BaseReplicaFetchTest.scala @@ -17,32 +17,39 @@ package kafka.server -import org.scalatest.junit.JUnit3Suite +import java.io.File + +import org.junit.{Test, After, Before} import kafka.zk.ZooKeeperTestHarness import kafka.utils.TestUtils._ import kafka.producer.KeyedMessage import kafka.serializer.StringEncoder -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,false) - val configs = props.map(p => new KafkaConfig(p)) +abstract class BaseReplicaFetchTest extends ZooKeeperTestHarness { var brokers: Seq[KafkaServer] = null val topic1 = "foo" val topic2 = "bar" + /* If this is `Some`, SSL will be enabled */ + protected def trustStoreFile: Option[File] + + @Before override def setUp() { super.setUp() - brokers = configs.map(config => TestUtils.createServer(config)) + brokers = createBrokerConfigs(2, zkConnect, enableControlledShutdown = false, enableSSL = trustStoreFile.isDefined, trustStoreFile = trustStoreFile) + .map(KafkaConfig.fromProps) + .map(TestUtils.createServer(_)) } + @After override def tearDown() { brokers.foreach(_.shutdown()) super.tearDown() } + @Test def testReplicaFetcherThread() { val partition = 0 val testMessageList1 = List("test1", "test2", "test3", "test4") @@ -54,7 +61,7 @@ class ReplicaFetchTest extends JUnit3Suite with ZooKeeperTestHarness { } // send test messages to leader - val producer = TestUtils.createProducer[String, String](TestUtils.getBrokerListStrFromConfigs(configs), + 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)) @@ -66,8 +73,9 @@ 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 } diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala new file mode 100644 index 0000000000000..75e856a0ba221 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala @@ -0,0 +1,160 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.Collections + +import org.apache.kafka.common.MetricName +import org.apache.kafka.common.metrics.{MetricConfig, Metrics, Quota} +import org.apache.kafka.common.utils.MockTime +import org.junit.{Assert, Before, Test} + +class ClientQuotaManagerTest { + private val time = new MockTime + + private val config = ClientQuotaManagerConfig(quotaBytesPerSecondDefault = 500, + quotaBytesPerSecondOverrides = "p1=2000,p2=4000") + + var numCallbacks: Int = 0 + def callback(delayTimeMs: Int) { + numCallbacks += 1 + } + + @Before + def beforeMethod() { + numCallbacks = 0 + } + + @Test + def testQuotaParsing() { + val clientMetrics = new ClientQuotaManager(config, newMetrics, "producer", time) + try { + Assert.assertEquals("Default producer quota should be 500", + new Quota(500, true), clientMetrics.quota("random-client-id")) + Assert.assertEquals("Should return the overridden value (2000)", + new Quota(2000, true), clientMetrics.quota("p1")) + Assert.assertEquals("Should return the overridden value (4000)", + new Quota(4000, true), clientMetrics.quota("p2")) + } finally { + clientMetrics.shutdown() + } + } + + @Test + def testQuotaViolation() { + val metrics = newMetrics + val clientMetrics = new ClientQuotaManager(config, metrics, "producer", time) + val queueSizeMetric = metrics.metrics().get(new MetricName("queue-size", "producer", "")) + try { + /* We have 10 second windows. Make sure that there is no quota violation + * if we produce under the quota + */ + for (i <- 0 until 10) { + clientMetrics.recordAndMaybeThrottle("unknown", 400, callback) + time.sleep(1000) + } + Assert.assertEquals(10, numCallbacks) + Assert.assertEquals(0, queueSizeMetric.value().toInt) + + // Create a spike. + // 400*10 + 2000 + 300 = 6300/10.5 = 600 bytes per second. + // (600 - quota)/quota*window-size = (600-500)/500*10.5 seconds = 2100 + // 10.5 seconds because the last window is half complete + time.sleep(500) + val sleepTime = clientMetrics.recordAndMaybeThrottle("unknown", 2300, callback) + + Assert.assertEquals("Should be throttled", 2100, sleepTime) + Assert.assertEquals(1, queueSizeMetric.value().toInt) + // After a request is delayed, the callback cannot be triggered immediately + clientMetrics.throttledRequestReaper.doWork() + Assert.assertEquals(10, numCallbacks) + time.sleep(sleepTime) + + // Callback can only be triggered after the the delay time passes + clientMetrics.throttledRequestReaper.doWork() + Assert.assertEquals(0, queueSizeMetric.value().toInt) + Assert.assertEquals(11, numCallbacks) + + // Could continue to see delays until the bursty sample disappears + for (i <- 0 until 10) { + clientMetrics.recordAndMaybeThrottle("unknown", 400, callback) + time.sleep(1000) + } + + Assert.assertEquals("Should be unthrottled since bursty sample has rolled over", + 0, clientMetrics.recordAndMaybeThrottle("unknown", 0, callback)) + } finally { + clientMetrics.shutdown() + } + } + + @Test + def testOverrideParse() { + var testConfig = ClientQuotaManagerConfig() + var clientMetrics = new ClientQuotaManager(testConfig, newMetrics, "consumer", time) + + try { + // Case 1 - Default config + Assert.assertEquals(new Quota(ClientQuotaManagerConfig.QuotaBytesPerSecondDefault, true), + clientMetrics.quota("p1")) + } finally { + clientMetrics.shutdown() + } + + + // Case 2 - Empty override + testConfig = ClientQuotaManagerConfig(quotaBytesPerSecondDefault = 500, + quotaBytesPerSecondOverrides = "p1=2000,p2=4000,,") + + clientMetrics = new ClientQuotaManager(testConfig, newMetrics, "consumer", time) + try { + Assert.assertEquals(new Quota(2000, true), clientMetrics.quota("p1")) + Assert.assertEquals(new Quota(4000, true), clientMetrics.quota("p2")) + } finally { + clientMetrics.shutdown() + } + + // Case 3 - NumberFormatException for override + testConfig = ClientQuotaManagerConfig(quotaBytesPerSecondDefault = 500, + quotaBytesPerSecondOverrides = "p1=2000,p2=4000,p3=p4") + try { + clientMetrics = new ClientQuotaManager(testConfig, newMetrics, "consumer", time) + Assert.fail("Should fail to parse invalid config " + testConfig.quotaBytesPerSecondOverrides) + } + catch { + // Swallow. + case nfe: NumberFormatException => + } + + // Case 4 - IllegalArgumentException for override + testConfig = ClientQuotaManagerConfig(quotaBytesPerSecondDefault = 500, + quotaBytesPerSecondOverrides = "p1=2000=3000") + try { + clientMetrics = new ClientQuotaManager(testConfig, newMetrics, "producer", time) + Assert.fail("Should fail to parse invalid config " + testConfig.quotaBytesPerSecondOverrides) + } + catch { + // Swallow. + case nfe: IllegalArgumentException => + } + + } + + def newMetrics: Metrics = { + new Metrics(new MetricConfig(), Collections.emptyList(), time) + } +} 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..df8d5b1b8642a --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala @@ -0,0 +1,128 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.{After, Before, Test} +import org.junit.Assert._ + +class DelayedOperationTest { + + var purgatory: DelayedOperationPurgatory[MockDelayedOperation] = null + + @Before + def setUp() { + purgatory = new DelayedOperationPurgatory[MockDelayedOperation](purgatoryName = "mock") + } + + @After + def tearDown() { + purgatory.shutdown() + } + + @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 index ad121169a5e80..16ac40d99f27e 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -16,45 +16,127 @@ */ package kafka.server +import java.util.Properties + import junit.framework.Assert._ +import org.easymock.{Capture, EasyMock} 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 { - - override val configs = List(new KafkaConfig(TestUtils.createBrokerConfig(0, TestUtils.choosePort))) +class DynamicConfigChangeTest extends KafkaServerTestHarness { + def generateConfigs() = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnect))) @Test def testConfigChange() { - val oldVal = 100000 - val newVal = 200000 + assertTrue("Should contain a ConfigHandler for topics", + this.servers(0).dynamicConfigHandlers.contains(ConfigType.Topic)) + val oldVal: java.lang.Long = 100000L + val newVal: java.lang.Long = 200000L val tp = TopicAndPartition("test", 0) - AdminUtils.createTopic(zkClient, tp.topic, 1, 1, LogConfig(flushInterval = oldVal).toProps) + 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) } - AdminUtils.changeTopicConfig(zkClient, tp.topic, LogConfig(flushInterval = newVal).toProps) + 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) } } + // For now client config changes do not do anything. Simply verify that the call was made + @Test + def testClientConfigChange() { + assertTrue("Should contain a ConfigHandler for topics", + this.servers(0).dynamicConfigHandlers.contains(ConfigType.Client)) + val clientId = "testClient" + val props = new Properties() + props.put("a.b", "c") + props.put("x.y", "z") + AdminUtils.changeClientIdConfig(zkClient, clientId, props) + TestUtils.retry(10000) { + val configHandler = this.servers(0).dynamicConfigHandlers(ConfigType.Client).asInstanceOf[ClientIdConfigHandler] + assertTrue("ClientId testClient must exist", configHandler.configPool.contains(clientId)) + assertEquals("ClientId testClient must be the only override", 1, configHandler.configPool.size) + assertEquals("c", configHandler.configPool.get(clientId).getProperty("a.b")) + assertEquals("z", configHandler.configPool.get(clientId).getProperty("x.y")) + } + } + @Test def testConfigChangeOnNonExistingTopic() { val topic = TestUtils.tempTopic try { - AdminUtils.changeTopicConfig(zkClient, topic, LogConfig(flushInterval = 10000).toProps) + 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 + @Test + def testProcessNotification { + val props = new Properties() + props.put("a.b", "10") + + // Create a mock ConfigHandler to record config changes it is asked to process + val entityArgument = new Capture[String]() + val propertiesArgument = new Capture[Properties]() + val handler = EasyMock.createNiceMock(classOf[ConfigHandler]) + handler.processConfigChanges( + EasyMock.and(EasyMock.capture(entityArgument), EasyMock.isA(classOf[String])), + EasyMock.and(EasyMock.capture(propertiesArgument), EasyMock.isA(classOf[Properties]))) + EasyMock.expectLastCall().once() + EasyMock.replay(handler) + + val configManager = new DynamicConfigManager(zkClient, Map(ConfigType.Topic -> handler)) + // Notifications created using the old TopicConfigManager are ignored. + configManager.processNotification(Some("not json")) + + // Incorrect Map. No version + try { + val jsonMap = Map("v" -> 1, "x" -> 2) + configManager.processNotification(Some(Json.encode(jsonMap))) + fail("Should have thrown an Exception while parsing incorrect notification " + jsonMap) + } + catch { + case t: Throwable => + } + // Version is provided. EntityType is incorrect + try { + val jsonMap = Map("version" -> 1, "entity_type" -> "garbage", "entity_name" -> "x") + configManager.processNotification(Some(Json.encode(jsonMap))) + fail("Should have thrown an Exception while parsing incorrect notification " + jsonMap) + } + catch { + case t: Throwable => + } + + // EntityName isn't provided + try { + val jsonMap = Map("version" -> 1, "entity_type" -> ConfigType.Topic) + configManager.processNotification(Some(Json.encode(jsonMap))) + fail("Should have thrown an Exception while parsing incorrect notification " + jsonMap) + } + catch { + case t: Throwable => + } + + // Everything is provided + val jsonMap = Map("version" -> 1, "entity_type" -> ConfigType.Topic, "entity_name" -> "x") + configManager.processNotification(Some(Json.encode(jsonMap))) + + // Verify that processConfigChanges was only called once + EasyMock.verify(handler) + } +} 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 03a424d45215e..c288e561c83e1 --- a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala +++ b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala @@ -19,18 +19,19 @@ package kafka.server import kafka.log._ import java.io.File import org.I0Itec.zkclient.ZkClient -import org.scalatest.junit.JUnit3Suite +import org.apache.kafka.common.metrics.Metrics import org.easymock.EasyMock 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 +import org.apache.kafka.common.utils.{MockTime => JMockTime} -class HighwatermarkPersistenceTest extends JUnit3Suite { +class HighwatermarkPersistenceTest { - 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 => TestUtils.createLogManager( @@ -41,9 +42,10 @@ class HighwatermarkPersistenceTest extends JUnit3Suite { @After def teardown() { for(manager <- logManagers; dir <- manager.logDirs) - Utils.rm(dir) + CoreUtils.rm(dir) } + @Test def testHighWatermarkPersistenceSinglePartition() { // mock zkclient val zkClient = EasyMock.createMock(classOf[ZkClient]) @@ -52,30 +54,40 @@ class HighwatermarkPersistenceTest extends JUnit3Suite { // create kafka scheduler val scheduler = new KafkaScheduler(2) scheduler.startup + val metrics = new Metrics // create replica manager - val replicaManager = new ReplicaManager(configs.head, new MockTime(), zkClient, scheduler, logManagers(0), new AtomicBoolean(false)) + val replicaManager = new ReplicaManager(configs.head, metrics, new MockTime, new JMockTime, zkClient, scheduler, + logManagers(0), new AtomicBoolean(false)) replicaManager.startup() - replicaManager.checkpointHighWatermarks() - var fooPartition0Hw = hwmFor(replicaManager, topic, 0) - assertEquals(0L, fooPartition0Hw) - 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)) - partition0.addReplicaIfNotExists(leaderReplicaPartition0) - val followerReplicaPartition0 = new Replica(configs.last.brokerId, partition0, SystemTime) - partition0.addReplicaIfNotExists(followerReplicaPartition0) - replicaManager.checkpointHighWatermarks() - fooPartition0Hw = hwmFor(replicaManager, topic, 0) - 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.messageOffset, fooPartition0Hw) - EasyMock.verify(zkClient) + try { + replicaManager.checkpointHighWatermarks() + var fooPartition0Hw = hwmFor(replicaManager, topic, 0) + assertEquals(0L, fooPartition0Hw) + 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)) + partition0.addReplicaIfNotExists(leaderReplicaPartition0) + val followerReplicaPartition0 = new Replica(configs.last.brokerId, partition0, SystemTime) + partition0.addReplicaIfNotExists(followerReplicaPartition0) + replicaManager.checkpointHighWatermarks() + fooPartition0Hw = hwmFor(replicaManager, topic, 0) + 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.messageOffset, fooPartition0Hw) + EasyMock.verify(zkClient) + } finally { + // shutdown the replica manager upon test completion + replicaManager.shutdown(false) + metrics.close() + scheduler.shutdown() + } } + @Test def testHighWatermarkPersistenceMultiplePartitions() { val topic1 = "foo1" val topic2 = "foo2" @@ -85,51 +97,60 @@ class HighwatermarkPersistenceTest extends JUnit3Suite { // create kafka scheduler val scheduler = new KafkaScheduler(2) scheduler.startup + val metrics = new Metrics // create replica manager - val replicaManager = new ReplicaManager(configs.head, new MockTime(), zkClient, scheduler, logManagers(0), new AtomicBoolean(false)) + val replicaManager = new ReplicaManager(configs.head, metrics, new MockTime(), new JMockTime, zkClient, + scheduler, logManagers(0), new AtomicBoolean(false)) replicaManager.startup() - replicaManager.checkpointHighWatermarks() - var topic1Partition0Hw = hwmFor(replicaManager, topic1, 0) - assertEquals(0L, topic1Partition0Hw) - val topic1Partition0 = replicaManager.getOrCreatePartition(topic1, 0) - // create leader log - val topic1Log0 = logManagers(0).createLog(TopicAndPartition(topic1, 0), LogConfig()) - // create a local replica for topic1 - val leaderReplicaTopic1Partition0 = new Replica(configs.head.brokerId, topic1Partition0, SystemTime, 0, Some(topic1Log0)) - topic1Partition0.addReplicaIfNotExists(leaderReplicaTopic1Partition0) - replicaManager.checkpointHighWatermarks() - topic1Partition0Hw = hwmFor(replicaManager, topic1, 0) - 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.messageOffset) - assertEquals(5L, topic1Partition0Hw) - // add another partition and set highwatermark - val topic2Partition0 = replicaManager.getOrCreatePartition(topic2, 0) - // create leader log - val topic2Log0 = logManagers(0).createLog(TopicAndPartition(topic2, 0), LogConfig()) - // create a local replica for topic2 - val leaderReplicaTopic2Partition0 = new Replica(configs.head.brokerId, topic2Partition0, SystemTime, 0, Some(topic2Log0)) - topic2Partition0.addReplicaIfNotExists(leaderReplicaTopic2Partition0) - replicaManager.checkpointHighWatermarks() - var topic2Partition0Hw = hwmFor(replicaManager, topic2, 0) - assertEquals(leaderReplicaTopic2Partition0.highWatermark.messageOffset, topic2Partition0Hw) - // set the highwatermark for local replica - topic2Partition0.getReplica().get.highWatermark = new LogOffsetMetadata(15L) - assertEquals(15L, leaderReplicaTopic2Partition0.highWatermark.messageOffset) - // change the highwatermark for topic1 - 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) - assertEquals(15L, topic2Partition0Hw) - // verify checkpointed hw for topic 1 - topic1Partition0Hw = hwmFor(replicaManager, topic1, 0) - assertEquals(10L, topic1Partition0Hw) - EasyMock.verify(zkClient) + try { + replicaManager.checkpointHighWatermarks() + var topic1Partition0Hw = hwmFor(replicaManager, topic1, 0) + assertEquals(0L, topic1Partition0Hw) + val topic1Partition0 = replicaManager.getOrCreatePartition(topic1, 0) + // create leader log + val topic1Log0 = logManagers(0).createLog(TopicAndPartition(topic1, 0), LogConfig()) + // create a local replica for topic1 + val leaderReplicaTopic1Partition0 = new Replica(configs.head.brokerId, topic1Partition0, SystemTime, 0, Some(topic1Log0)) + topic1Partition0.addReplicaIfNotExists(leaderReplicaTopic1Partition0) + replicaManager.checkpointHighWatermarks() + topic1Partition0Hw = hwmFor(replicaManager, topic1, 0) + 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.messageOffset) + assertEquals(5L, topic1Partition0Hw) + // add another partition and set highwatermark + val topic2Partition0 = replicaManager.getOrCreatePartition(topic2, 0) + // create leader log + val topic2Log0 = logManagers(0).createLog(TopicAndPartition(topic2, 0), LogConfig()) + // create a local replica for topic2 + val leaderReplicaTopic2Partition0 = new Replica(configs.head.brokerId, topic2Partition0, SystemTime, 0, Some(topic2Log0)) + topic2Partition0.addReplicaIfNotExists(leaderReplicaTopic2Partition0) + replicaManager.checkpointHighWatermarks() + var topic2Partition0Hw = hwmFor(replicaManager, topic2, 0) + assertEquals(leaderReplicaTopic2Partition0.highWatermark.messageOffset, topic2Partition0Hw) + // set the highwatermark for local replica + topic2Partition0.getReplica().get.highWatermark = new LogOffsetMetadata(15L) + assertEquals(15L, leaderReplicaTopic2Partition0.highWatermark.messageOffset) + // change the highwatermark for topic1 + 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) + assertEquals(15L, topic2Partition0Hw) + // verify checkpointed hw for topic 1 + topic1Partition0Hw = hwmFor(replicaManager, topic1, 0) + assertEquals(10L, topic1Partition0Hw) + EasyMock.verify(zkClient) + } finally { + // shutdown the replica manager upon test completion + replicaManager.shutdown(false) + metrics.close() + scheduler.shutdown() + } } def hwmFor(replicaManager: ReplicaManager, topic: String, partition: Int): Long = { diff --git a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala index cd302aa51eb83..89a8fd9dc8b48 100644 --- a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala +++ b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala @@ -16,7 +16,10 @@ */ package kafka.server -import org.scalatest.junit.JUnit3Suite +import java.util.Properties + +import org.apache.kafka.common.metrics.Metrics +import org.junit.{Test, Before, After} import collection.mutable.HashMap import collection.mutable.Map import kafka.cluster.{Partition, Replica} @@ -25,19 +28,44 @@ import kafka.log.Log import org.junit.Assert._ import kafka.utils._ import java.util.concurrent.atomic.AtomicBoolean +import kafka.message.MessageSet +import org.apache.kafka.common.utils.{MockTime => JMockTime} + -class IsrExpirationTest extends JUnit3Suite { +class IsrExpirationTest { 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 + val jTime = new JMockTime + val metrics = new Metrics + + var replicaManager: ReplicaManager = null + + @Before + def setUp() { + replicaManager = new ReplicaManager(configs.head, metrics, time, jTime, null, null, null, new AtomicBoolean(false)) + } + + @After + def tearDown() { + replicaManager.shutdown(false) + metrics.close() + } + + /* + * 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 + */ + @Test def testIsrExpirationForStuckFollowers() { - val time = new MockTime val log = getLogWithLogEndOffset(15L, 2) // set logEndOffset for leader to 15L // create one partition and all replicas @@ -45,44 +73,92 @@ 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 = new LogOffsetMetadata(10L)) - 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 + */ + @Test + 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 + */ + @Test 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 = new LogOffsetMetadata(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) val leaderReplica = new Replica(leaderId, partition, time, 0, Some(localLog)) @@ -108,4 +184,4 @@ class IsrExpirationTest extends JUnit3Suite { new Replica(config.brokerId, partition, time) } } -} \ No newline at end of file +} 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 2377abe4933e0..bfec4266c0cec --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -17,169 +17,544 @@ package kafka.server -import org.junit.Test +import java.util.Properties + import junit.framework.Assert._ -import org.scalatest.junit.JUnit3Suite -import kafka.utils.TestUtils +import kafka.api.{ApiVersion, KAFKA_082} +import kafka.message._ +import kafka.utils.{TestUtils, CoreUtils} +import org.apache.kafka.common.config.ConfigException +import org.apache.kafka.common.protocol.SecurityProtocol +import org.junit.{Assert, Test} +import org.scalatest.Assertions.intercept -class KafkaConfigTest extends JUnit3Suite { +class KafkaConfigTest { @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, 8181) - props.put("log.retention.ms", "1800000") + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + props.put(KafkaConfig.LogRetentionTimeMillisProp, "1800000") - val cfg = new KafkaConfig(props) + 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 = new KafkaConfig(props) + val cfg = KafkaConfig.fromProps(props) assertEquals( 30 * 60L * 1000L, cfg.logRetentionTimeMillis) - } - + @Test def testLogRetentionTimeBothMinutesAndMsProvided() { - val props = TestUtils.createBrokerConfig(0, 8181) - props.put("log.retention.ms", "1800000") - props.put("log.retention.minutes", "10") + 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 props = TestUtils.createBrokerConfig(0, port) - props.put("advertised.host.name", advertisedHostName) - props.put("advertised.port", advertisedPort.toString) - - val serverConfig = new KafkaConfig(props) - - assertEquals(serverConfig.advertisedHostName, advertisedHostName) - assertEquals(serverConfig.advertisedPort, advertisedPort) + 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 + + assertEquals(endpoint.host, advertisedHostName) + assertEquals(endpoint.port, advertisedPort.toInt) + } + + @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) + } + + @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, 8181) - val serverConfig = new KafkaConfig(props) + 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, 8181) - props.put("unclean.leader.election.enable", String.valueOf(false)) - val serverConfig = new KafkaConfig(props) + 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, 8181) - props.put("unclean.leader.election.enable", String.valueOf(true)) - val serverConfig = new KafkaConfig(props) + 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, 8181) - props.put("unclean.leader.election.enable", "invalid") + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + props.put(KafkaConfig.UncleanLeaderElectionEnableProp, "invalid") - intercept[IllegalArgumentException] { - new KafkaConfig(props) + intercept[ConfigException] { + KafkaConfig.fromProps(props) } } - + @Test def testLogRollTimeMsProvided() { - val props = TestUtils.createBrokerConfig(0, 8181) - props.put("log.roll.ms", "1800000") + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + props.put(KafkaConfig.LogRollTimeMillisProp, "1800000") - val cfg = new KafkaConfig(props) + val cfg = KafkaConfig.fromProps(props) assertEquals(30 * 60L * 1000L, cfg.logRollTimeMillis) - } - + @Test def testLogRollTimeBothMsAndHoursProvided() { - val props = TestUtils.createBrokerConfig(0, 8181) - props.put("log.roll.ms", "1800000") - props.put("log.roll.hours", "1") + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + props.put(KafkaConfig.LogRollTimeMillisProp, "1800000") + props.put(KafkaConfig.LogRollTimeHoursProp, "1") - val cfg = new KafkaConfig(props) + val cfg = KafkaConfig.fromProps(props) assertEquals( 30 * 60L * 1000L, cfg.logRollTimeMillis) - } - + @Test def testLogRollTimeNoConfigProvided() { - 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.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) + } + } + + @Test + def testFromPropsInvalid() { + def getBaseProperties(): Properties = { + val validRequiredProperties = new Properties() + validRequiredProperties.put(KafkaConfig.ZkConnectProp, "127.0.0.1:2181") + 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.RequestTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + + case KafkaConfig.AuthorizerClassNameProp => //ignore string + + 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.ProducerQuotaBytesPerSecondDefaultProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.ConsumerQuotaBytesPerSecondDefaultProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.ProducerQuotaBytesPerSecondOverridesProp => // ignore string + case KafkaConfig.ConsumerQuotaBytesPerSecondOverridesProp => // ignore string + case KafkaConfig.NumQuotaSamplesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.QuotaWindowSizeSecondsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + + 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 + + //SSL Configs + case KafkaConfig.PrincipalBuilderClassProp => + case KafkaConfig.SSLProtocolProp => // ignore string + case KafkaConfig.SSLProviderProp => // ignore string + case KafkaConfig.SSLEnabledProtocolsProp => + case KafkaConfig.SSLKeystoreTypeProp => // ignore string + case KafkaConfig.SSLKeystoreLocationProp => // ignore string + case KafkaConfig.SSLKeystorePasswordProp => // ignore string + case KafkaConfig.SSLKeyPasswordProp => // ignore string + case KafkaConfig.SSLTruststoreTypeProp => // ignore string + case KafkaConfig.SSLTruststorePasswordProp => // ignore string + case KafkaConfig.SSLTruststoreLocationProp => // ignore string + case KafkaConfig.SSLKeyManagerAlgorithmProp => + case KafkaConfig.SSLTrustManagerAlgorithmProp => + case KafkaConfig.SSLClientAuthProp => // 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) + } + }) } - } 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 c2ba07c5fdbaf..0efaa6ac51176 --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -17,43 +17,53 @@ package kafka.server -import org.scalatest.junit.JUnit3Suite -import kafka.zk.ZooKeeperTestHarness -import kafka.utils.TestUtils._ -import junit.framework.Assert._ -import kafka.utils.{ZkUtils, Utils, TestUtils} -import kafka.controller.{ControllerContext, LeaderIsrAndControllerEpoch, ControllerChannelManager} +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.requests.LeaderAndIsrRequest.PartitionState + +import scala.collection.JavaConverters._ +import kafka.api.{PartitionStateInfo, LeaderAndIsr} +import org.apache.kafka.common.requests.{LeaderAndIsrResponse, LeaderAndIsrRequest, AbstractRequestResponse} +import org.junit.Assert._ +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.metrics.Metrics +import org.apache.kafka.common.protocol.{ApiKeys, SecurityProtocol} +import org.apache.kafka.common.utils.SystemTime +import org.junit.{Test, After, Before} -class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { +class LeaderElectionTest extends ZooKeeperTestHarness { val brokerId1 = 0 val brokerId2 = 1 - val port1 = TestUtils.choosePort() - val port2 = TestUtils.choosePort() - - val configProps1 = TestUtils.createBrokerConfig(brokerId1, port1, false) - val configProps2 = TestUtils.createBrokerConfig(brokerId2, port2, false) var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] var staleControllerEpochDetected = false + @Before 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) } + @After 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() } + @Test def testLeaderElectionAndEpoch { // start 2 brokers val topic = "new-topic" @@ -99,6 +109,7 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals("Second epoch value should be %d".format(leaderEpoch2+1) , leaderEpoch2+1, leaderEpoch3) } + @Test def testLeaderElectionWithStaleControllerEpoch() { // start 2 brokers val topic = "new-topic" @@ -117,29 +128,41 @@ 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 controllerContext = new ControllerContext(zkClient, 6000) + + 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 => + val brokerEndPoint = b.getBrokerEndPoint(SecurityProtocol.PLAINTEXT) + new LeaderAndIsrRequest.EndPoint(brokerEndPoint.id, brokerEndPoint.host, brokerEndPoint.port) + } + + val controllerContext = new ControllerContext(zkClient, zkConnection, 6000) controllerContext.liveBrokers = brokers.toSet - val controllerChannelManager = new ControllerChannelManager(controllerContext, controllerConfig) + val metrics = new Metrics + val controllerChannelManager = new ControllerChannelManager(controllerContext, controllerConfig, new SystemTime, metrics) controllerChannelManager.startup() - val staleControllerEpoch = 0 - val leaderAndIsr = new collection.mutable.HashMap[(String, Int), LeaderIsrAndControllerEpoch] - leaderAndIsr.put((topic, partitionId), - new LeaderIsrAndControllerEpoch(new LeaderAndIsr(brokerId2, List(brokerId1, brokerId2)), 2)) - val partitionStateInfo = leaderAndIsr.mapValues(l => new PartitionStateInfo(l, Set(0,1))).toMap - val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfo, brokers.toSet, controllerId, - staleControllerEpoch, 0, "") - - controllerChannelManager.sendRequest(brokerId2, leaderAndIsrRequest, staleControllerEpochCallback) - TestUtils.waitUntilTrue(() => staleControllerEpochDetected == true, - "Controller epoch should be stale") - assertTrue("Stale controller epoch not detected by the broker", staleControllerEpochDetected) - - controllerChannelManager.shutdown() + try { + val staleControllerEpoch = 0 + val partitionStates = Map( + new TopicPartition(topic, partitionId) -> new PartitionState(2, brokerId2, LeaderAndIsr.initialLeaderEpoch, + Seq(brokerId1, brokerId2).map(Integer.valueOf).asJava, LeaderAndIsr.initialZKVersion, + Set(0, 1).map(Integer.valueOf).asJava) + ) + val leaderAndIsrRequest = new LeaderAndIsrRequest(controllerId, staleControllerEpoch, partitionStates.asJava, + brokerEndPoints.toSet.asJava) + + controllerChannelManager.sendRequest(brokerId2, ApiKeys.LEADER_AND_ISR, None, leaderAndIsrRequest, + staleControllerEpochCallback) + TestUtils.waitUntilTrue(() => staleControllerEpochDetected == true, + "Controller epoch should be stale") + assertTrue("Stale controller epoch not detected by the broker", staleControllerEpochDetected) + } finally { + controllerChannelManager.shutdown() + metrics.close() + } } - private def staleControllerEpochCallback(response: RequestOrResponse): Unit = { + private def staleControllerEpochCallback(response: AbstractRequestResponse): Unit = { val leaderAndIsrResponse = response.asInstanceOf[LeaderAndIsrResponse] staleControllerEpochDetected = leaderAndIsrResponse.errorCode match { case ErrorMapping.StaleControllerEpochCode => true 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 9556ed92c61ff..344001d25b01b --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -19,12 +19,11 @@ package kafka.server import java.io.File import kafka.utils._ -import junit.framework.Assert._ +import org.junit.Assert._ import java.util.{Random, Properties} import kafka.consumer.SimpleConsumer import kafka.message.{NoCompressionCodec, ByteBufferMessageSet, Message} import kafka.zk.ZooKeeperTestHarness -import org.scalatest.junit.JUnit3Suite import kafka.admin.AdminUtils import kafka.api.{PartitionOffsetRequestInfo, FetchRequestBuilder, OffsetRequest} import kafka.utils.TestUtils._ @@ -33,32 +32,31 @@ import org.junit.After import org.junit.Before import org.junit.Test -class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { +class LogOffsetTest extends ZooKeeperTestHarness { val random = new Random() var logDir: File = null var topicLogDir: File = null var server: KafkaServer = null var logSize: Int = 100 - val brokerPort: Int = 9099 var simpleConsumer: SimpleConsumer = null var time: Time = new MockTime() @Before override def setUp() { super.setUp() - val config: Properties = createBrokerConfig(1, brokerPort) + val 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() } @@ -92,7 +90,7 @@ 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), "Leader should be elected") val topicAndPartition = TopicAndPartition(topic, part) @@ -101,7 +99,7 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { 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( @@ -155,14 +153,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), "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 @@ -194,10 +192,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 d5d351c4f2593..46829b818acb5 --- a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala @@ -16,9 +16,10 @@ */ package kafka.server +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.{KeyedMessage, Producer} @@ -26,59 +27,77 @@ import kafka.serializer.StringEncoder import java.io.File -import org.scalatest.junit.JUnit3Suite +import org.junit.{Test, After, Before} import org.junit.Assert._ -class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { +class LogRecoveryTest extends ZooKeeperTestHarness { + + 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) - val configs = TestUtils.createBrokerConfigs(2, false).map(new KafkaConfig(_) { - override val replicaLagTimeMaxMs = 5000L - override val replicaLagMaxMessages = 10L - override val replicaFetchWaitMaxMs = 1000 - override val replicaFetchMinBytes = 20 - }) + 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] + // 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) + } + + @Before 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) + servers = List(server1, server2) // create topic with 1 partition, 2 replicas, one on each broker createTopic(zkClient, topic, partitionReplicaAssignment = Map(0->Seq(0,1)), servers = servers) // create the producer - producer = TestUtils.createProducer[Int, String](TestUtils.getBrokerListStrFromConfigs(configs), - encoder = classOf[StringEncoder].getName, - keyEncoder = classOf[IntEncoder].getName) + updateProducer() } + @After override def tearDown() { producer.close() for(server <- servers) { server.shutdown() - Utils.rm(server.config.logDirs(0)) + CoreUtils.rm(server.config.logDirs(0)) } super.tearDown() } + @Test def testHWCheckpointNoFailuresSingleLogSegment { val numMessages = 2L sendMessages(numMessages.toInt) @@ -95,6 +114,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(numMessages, followerHW) } + @Test def testHWCheckpointWithFailuresSingleLogSegment { var leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId) @@ -114,6 +134,8 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { // bring the preferred replica back server1.startup() + // Update producer with new server settings + updateProducer() leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId) assertTrue("Leader must remain on broker 1, in case of zookeeper session expiration it can move to broker 0", @@ -125,6 +147,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L)) server2.startup() + 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)) @@ -142,6 +165,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L)) } + @Test def testHWCheckpointNoFailuresMultipleLogSegments { sendMessages(20) val hw = 20L @@ -157,6 +181,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(hw, followerHW) } + @Test def testHWCheckpointWithFailuresMultipleLogSegments { var leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId) @@ -174,6 +199,7 @@ 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, oldLeaderOpt = leader) assertEquals("Leader must move to broker 1", 1, leader.getOrElse(-1)) @@ -182,6 +208,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { // 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)) @@ -189,6 +216,9 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { sendMessages(2) hw += 2 + // allow some time for the follower to create replica + TestUtils.waitUntilTrue(() => server1.replicaManager.getReplica(topic, 0).nonEmpty, + "Failed to create replica in follower after timeout") // allow some time for the follower to get the leader HW TestUtils.waitUntilTrue(() => server1.replicaManager.getReplica(topic, 0).get.highWatermark.messageOffset == hw, 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 2d9325045ac1a..f8466985fc44d --- 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,40 +17,45 @@ package kafka.server -import java.io.File -import kafka.utils._ -import junit.framework.Assert._ -import java.util.Properties -import kafka.consumer.SimpleConsumer -import org.junit.{After, Before, Test} -import kafka.zk.ZooKeeperTestHarness -import org.scalatest.junit.JUnit3Suite import kafka.api.{ConsumerMetadataRequest, OffsetCommitRequest, OffsetFetchRequest} +import kafka.consumer.SimpleConsumer +import kafka.common.{OffsetMetadata, OffsetMetadataAndError, OffsetAndMetadata, ErrorMapping, TopicAndPartition} +import kafka.utils._ import kafka.utils.TestUtils._ -import kafka.common.{OffsetMetadataAndError, OffsetAndMetadata, ErrorMapping, TopicAndPartition} +import kafka.zk.ZooKeeperTestHarness + +import org.junit.{After, Before, Test} + +import java.util.Properties +import java.io.File + import scala.util.Random import scala.collection._ -class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { +import org.junit.Assert._ + +class OffsetCommitTest extends 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 - val group = "test-group" 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) @@ -65,7 +70,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { override def tearDown() { simpleConsumer.close server.shutdown - Utils.rm(logDir) + CoreUtils.rm(logDir) super.tearDown() } @@ -79,7 +84,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { // create the topic createTopic(zkClient, topic, partitionReplicaAssignment = expectedReplicaAssignment, servers = Seq(server)) - val commitRequest = OffsetCommitRequest("test-group", immutable.Map(topicAndPartition -> OffsetAndMetadata(offset=42L))) + val commitRequest = OffsetCommitRequest(group, immutable.Map(topicAndPartition -> OffsetAndMetadata(offset = 42L))) val commitResponse = simpleConsumer.commitOffsets(commitRequest) assertEquals(ErrorMapping.NoError, commitResponse.commitStatus.get(topicAndPartition).get) @@ -89,7 +94,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { val fetchResponse = simpleConsumer.fetchOffsets(fetchRequest) assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(topicAndPartition).get.error) - assertEquals(OffsetAndMetadata.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 @@ -109,6 +114,13 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { 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 @@ -116,7 +128,13 @@ 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 + + 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"), @@ -136,30 +154,43 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { 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.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(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.metadata) - assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.metadata) - assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 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(OffsetAndMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.offset) - assertEquals(OffsetAndMetadata.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 @@ -184,6 +215,95 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { val commitResponse1 = simpleConsumer.commitOffsets(commitRequest1) 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 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)) + 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.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/PlaintextReplicaFetchTest.scala b/core/src/test/scala/unit/kafka/server/PlaintextReplicaFetchTest.scala new file mode 100644 index 0000000000000..871e49b6f8d83 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/PlaintextReplicaFetchTest.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.server + +class PlaintextReplicaFetchTest extends BaseReplicaFetchTest { + protected def trustStoreFile = None +} diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index a9c4ddc78df0b..181334919cdca 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -17,44 +17,93 @@ 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.metrics.Metrics +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.utils.{MockTime => JMockTime} import org.easymock.EasyMock import org.I0Itec.zkclient.ZkClient -import org.scalatest.junit.JUnit3Suite import org.junit.Test -class ReplicaManagerTest extends JUnit3Suite { +import scala.collection.Map + +class ReplicaManagerTest { val topic = "test-topic" @Test def testHighWaterMarkDirectoryMapping() { - val props = TestUtils.createBrokerConfig(1) - val config = new KafkaConfig(props) + 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() + val jTime = new JMockTime + val metrics = new Metrics + val rm = new ReplicaManager(config, metrics, time, jTime, zkClient, new MockScheduler(time), mockLogMgr, + new AtomicBoolean(false)) + try { + val partition = rm.getOrCreatePartition(topic, 1) + partition.getOrCreateReplica(1) + rm.checkpointHighWatermarks() + } finally { + // shutdown the replica manager upon test completion + rm.shutdown(false) + metrics.close() + } } @Test def testHighwaterMarkRelativeDirectoryMapping() { - val props = TestUtils.createBrokerConfig(1) + val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect) props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath) - val config = new KafkaConfig(props) + 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 jTime = new JMockTime + val metrics = new Metrics + val rm = new ReplicaManager(config, metrics, time, jTime, zkClient, new MockScheduler(time), mockLogMgr, + new AtomicBoolean(false)) + try { + val partition = rm.getOrCreatePartition(topic, 1) + partition.getOrCreateReplica(1) + rm.checkpointHighWatermarks() + } finally { + // shutdown the replica manager upon test completion + rm.shutdown(false) + metrics.close() + } + } + + @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 partition = rm.getOrCreatePartition(topic, 1) - partition.getOrCreateReplica(1) - rm.checkpointHighWatermarks() + val jTime = new JMockTime + val metrics = new Metrics + val rm = new ReplicaManager(config, metrics, time, jTime, zkClient, new MockScheduler(time), mockLogMgr, + new AtomicBoolean(false), Option(this.getClass.getName)) + try { + 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) + } finally { + rm.shutdown(false) + metrics.close() + } + + TestUtils.verifyNonDaemonThreadsStatus(this.getClass.getName) } } 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 a577f4a8bf420..0000000000000 --- a/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala +++ /dev/null @@ -1,123 +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(5) - } - - 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").size) - assertFalse("r1 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r1)) - assertEquals("Still nothing satisfied", 0, purgatory.update("test1").size) - assertFalse("r2 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r2)) - assertEquals("Still nothing satisfied", 0, purgatory.update("test2").size) - purgatory.satisfied += r1 - assertEquals("r1 satisfied", mutable.ArrayBuffer(r1), purgatory.update("test1")) - assertEquals("Nothing satisfied", 0, purgatory.update("test1").size) - purgatory.satisfied += r2 - assertEquals("r2 satisfied", mutable.ArrayBuffer(r2), purgatory.update("test2")) - assertEquals("Nothing satisfied", 0, purgatory.update("test2").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 - assertFalse("r1 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r1)) - assertFalse("r2 not satisfied and hence watched", purgatory.checkAndMaybeWatch(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) - } - - @Test - def testRequestPurge() { - val r1 = new DelayedRequest(Array("test1"), null, 100000L) - val r12 = new DelayedRequest(Array("test1", "test2"), null, 100000L) - val r23 = new DelayedRequest(Array("test2", "test3"), null, 100000L) - purgatory.checkAndMaybeWatch(r1) - purgatory.checkAndMaybeWatch(r12) - purgatory.checkAndMaybeWatch(r23) - - assertEquals("Purgatory should have 5 watched elements", 5, purgatory.watched()) - assertEquals("Purgatory should have 3 total delayed requests", 3, purgatory.delayed()) - - // satisfy one of the requests, it should then be purged from the watch list with purge interval 5 - r12.satisfied.set(true) - TestUtils.waitUntilTrue(() => purgatory.watched() == 3, - "Purgatory should have 3 watched elements instead of " + + purgatory.watched(), 1000L) - TestUtils.waitUntilTrue(() => purgatory.delayed() == 3, - "Purgatory should still have 3 total delayed requests instead of " + purgatory.delayed(), 1000L) - - // add two more requests, then the satisfied request should be purged from the delayed queue with purge interval 5 - purgatory.checkAndMaybeWatch(r1) - purgatory.checkAndMaybeWatch(r1) - - TestUtils.waitUntilTrue(() => purgatory.watched() == 5, - "Purgatory should have 5 watched elements instead of " + purgatory.watched(), 1000L) - TestUtils.waitUntilTrue(() => purgatory.delayed() == 4, - "Purgatory should have 4 total delayed requests instead of " + purgatory.delayed(), 1000L) - } - - class MockRequestPurgatory(purge: Int) extends RequestPurgatory[DelayedRequest](purgeInterval = purge) { - val satisfied = mutable.Set[DelayedRequest]() - val expired = mutable.Set[DelayedRequest]() - def awaitExpiration(delayed: DelayedRequest) = { - delayed synchronized { - delayed.wait() - } - } - def checkSatisfied(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..9afb2caee27a9 --- /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.{Before, Test} +import org.junit.Assert._ +import java.io.File + +class ServerGenerateBrokerIdTest extends ZooKeeperTestHarness { + var props1: Properties = null + var config1: KafkaConfig = null + var props2: Properties = null + var config2: KafkaConfig = null + val brokerMetaPropsFile = "meta.properties" + + @Before + 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, threadNamePrefix = Option(this.getClass.getName)) + 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(this.getClass.getName) + } + + @Test + def testUserConfigAndGeneratedBrokerId() { + // start the server with broker.id as part of config + val server1 = new KafkaServer(config1, threadNamePrefix = Option(this.getClass.getName)) + val server2 = new KafkaServer(config2, threadNamePrefix = Option(this.getClass.getName)) + 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(this.getClass.getName) + } + + @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, threadNamePrefix = Option(this.getClass.getName)) + 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, threadNamePrefix = Option(this.getClass.getName)) + server1.startup() + server1.shutdown() + assertTrue(verifyBrokerMetadata(config1.logDirs, 1001)) + CoreUtils.rm(server1.config.logDirs) + TestUtils.verifyNonDaemonThreadsStatus(this.getClass.getName) + } + + @Test + def testConsistentBrokerIdFromUserConfigAndMetaProps() { + // check if configured brokerId and stored brokerId are equal or throw InconsistentBrokerException + var server1 = new KafkaServer(config1, threadNamePrefix = Option(this.getClass.getName)) //auto generate broker Id + server1.startup() + server1.shutdown() + server1 = new KafkaServer(config2, threadNamePrefix = Option(this.getClass.getName)) // user specified broker id + try { + server1.startup() + } catch { + case e: kafka.common.InconsistentBrokerIdException => //success + } + server1.shutdown() + CoreUtils.rm(server1.config.logDirs) + TestUtils.verifyNonDaemonThreadsStatus(this.getClass.getName) + } + + 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 3804a114e97c8..b6d5697dc3c3d --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -19,7 +19,7 @@ package kafka.server import kafka.zk.ZooKeeperTestHarness import kafka.consumer.SimpleConsumer import kafka.producer._ -import kafka.utils.{IntEncoder, TestUtils, Utils} +import kafka.utils.{IntEncoder, TestUtils, CoreUtils} import kafka.utils.TestUtils._ import kafka.api.FetchRequestBuilder import kafka.message.ByteBufferMessageSet @@ -27,25 +27,28 @@ import kafka.serializer.StringEncoder import java.io.File -import org.junit.Test -import org.scalatest.junit.JUnit3Suite -import junit.framework.Assert._ - -class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { - val port = TestUtils.choosePort - val props = TestUtils.createBrokerConfig(0, port) - val config = new KafkaConfig(props) +import org.junit.{Before, Test} +import org.junit.Assert._ +class ServerShutdownTest extends ZooKeeperTestHarness { + var config: KafkaConfig = null val host = "localhost" val topic = "test" val sent1 = List("hello", "there") val sent2 = List("more", "messages") + @Before + override def setUp() { + super.setUp() + val props = TestUtils.createBrokerConfig(0, zkConnect) + config = KafkaConfig.fromProps(props) + } + @Test def testCleanShutdown() { - var server = new KafkaServer(config) + var server = new KafkaServer(config, threadNamePrefix = Option(this.getClass.getName)) server.startup() - var producer = TestUtils.createProducer[Int, String](TestUtils.getBrokerListStrFromConfigs(Seq(config)), + var producer = TestUtils.createProducer[Int, String](TestUtils.getBrokerListStrFromServers(Seq(server)), encoder = classOf[StringEncoder].getName, keyEncoder = classOf[IntEncoder].getName) @@ -71,17 +74,17 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { // wait for the broker to receive the update metadata request after startup TestUtils.waitUntilMetadataIsPropagated(Seq(server), topic, 0) - producer = TestUtils.createProducer[Int, String](TestUtils.getBrokerListStrFromConfigs(Seq(config)), + producer = TestUtils.createProducer[Int, String](TestUtils.getBrokerListStrFromServers(Seq(server)), encoder = classOf[StringEncoder].getName, keyEncoder = classOf[IntEncoder].getName) - val consumer = new SimpleConsumer(host, port, 1000000, 64*1024, "") + 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 @@ -92,34 +95,34 @@ 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, port) + val newProps = TestUtils.createBrokerConfig(0, zkConnect) newProps.setProperty("delete.topic.enable", "true") - val newConfig = new KafkaConfig(newProps) - var server = new KafkaServer(newConfig) + val newConfig = KafkaConfig.fromProps(newProps) + val server = new KafkaServer(newConfig, threadNamePrefix = Option(this.getClass.getName)) server.startup() server.shutdown() server.awaitShutdown() - Utils.rm(server.config.logDirs) + CoreUtils.rm(server.config.logDirs) verifyNonDaemonThreadsStatus } @Test def testCleanShutdownAfterFailedStartup() { - val newProps = TestUtils.createBrokerConfig(0, port) + val newProps = TestUtils.createBrokerConfig(0, zkConnect) newProps.setProperty("zookeeper.connect", "fakehostthatwontresolve:65535") - val newConfig = new KafkaConfig(newProps) - var server = new KafkaServer(newConfig) + val newConfig = KafkaConfig.fromProps(newProps) + val server = new KafkaServer(newConfig, threadNamePrefix = Option(this.getClass.getName)) try { server.startup() fail("Expected KafkaServer setup to fail, throw exception") @@ -129,21 +132,41 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { // 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(server.brokerState.currentState, NotRunning.state) - if (server.brokerState.currentState != NotRunning.state) - server.shutdown() + assertEquals(NotRunning.state, server.brokerState.currentState) case e: Throwable => - fail("Expected KafkaServer setup to fail with connection exception but caught a different exception.") + 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() } - server.awaitShutdown() - Utils.rm(server.config.logDirs) + CoreUtils.rm(server.config.logDirs) verifyNonDaemonThreadsStatus } + private[this] def isNonDaemonKafkaThread(t: Thread): Boolean = { + !t.isDaemon && t.isAlive && t.getName.startsWith(this.getClass.getName) + } + def verifyNonDaemonThreadsStatus() { assertEquals(0, Thread.getAllStackTraces.keySet().toArray - .map(_.asInstanceOf[Thread]) - .count(t => !t.isDaemon && t.isAlive && t.getClass.getCanonicalName.toLowerCase.startsWith("kafka"))) + .map{ _.asInstanceOf[Thread] } + .count(isNonDaemonKafkaThread)) + } + + @Test + 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 old mode 100644 new mode 100755 index a0ed4855f2550..0adc0aa394242 --- a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala @@ -17,38 +17,55 @@ package kafka.server -import org.scalatest.junit.JUnit3Suite -import kafka.zk import kafka.utils.ZkUtils -import kafka.utils.Utils +import kafka.utils.CoreUtils import kafka.utils.TestUtils import kafka.zk.ZooKeeperTestHarness -import junit.framework.Assert._ +import org.junit.Assert._ +import org.junit.Test -class ServerStartupTest extends JUnit3Suite with ZooKeeperTestHarness { - var server : KafkaServer = null - val brokerId = 0 - val zookeeperChroot = "/kafka-chroot-for-unittest" +class ServerStartupTest extends ZooKeeperTestHarness { - override def setUp() { - super.setUp() - val props = TestUtils.createBrokerConfig(brokerId, TestUtils.choosePort()) + @Test + 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)) - server = TestUtils.createServer(new KafkaConfig(props)) - } + val pathExists = ZkUtils.pathExists(zkClient, zookeeperChroot) + assertTrue(pathExists) - override def tearDown() { server.shutdown() - Utils.rm(server.config.logDirs) - super.tearDown() + CoreUtils.rm(server.config.logDirs) } - def testBrokerCreatesZKChroot { - val pathExists = ZkUtils.pathExists(zkClient, zookeeperChroot) - assertTrue(pathExists) - } + @Test + 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 09ed8f5a7a414..0485f7b393c08 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -17,235 +17,142 @@ package kafka.server import kafka.api._ -import kafka.cluster.{Partition, Replica} -import kafka.common.{ErrorMapping, TopicAndPartition} +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 org.apache.kafka.common.metrics.Metrics +import org.apache.kafka.common.utils.{MockTime => JMockTime} +import org.junit.{Test, After, Before} -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 org.junit.Assert._ -class SimpleFetchTest extends JUnit3Suite { +class SimpleFetchTest { - val configs = TestUtils.createBrokerConfigs(2).map(new KafkaConfig(_) { - override val replicaLagTimeMaxMs = 100L - override val replicaFetchWaitMaxMs = 100 - override val replicaLagMaxMessages = 10L - }) - val topic = "foo" - val partitionId = 0 - - /** - * The scenario for this test is that there is one topic, "test-topic", 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 should only see data up to the HW of the leader, - * in this case up an offset of "5". - */ - def testNonReplicaSeesHwWhenFetching() { - /* setup */ - val time = new MockTime - val leo = 20L - val hw = 5 - val fetchSize = 100 - val messages = new Message("test-message".getBytes()) + val replicaLagTimeMaxMs = 100L + val replicaFetchWaitMaxMs = 100 + val replicaLagMaxMessages = 10L - // 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 overridingProps = new Properties() + overridingProps.put(KafkaConfig.ReplicaLagTimeMaxMsProp, replicaLagTimeMaxMs.toString) + overridingProps.put(KafkaConfig.ReplicaFetchWaitMaxMsProp, replicaFetchWaitMaxMs.toString) - 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 FetchDataInfo( - new LogOffsetMetadata(0L, 0L, leo.toInt), - new ByteBufferMessageSet(messages) - )).anyTimes() - EasyMock.replay(log) + val configs = TestUtils.createBrokerConfigs(2, TestUtils.MockZkConnect).map(KafkaConfig.fromProps(_, overridingProps)) - val logManager = EasyMock.createMock(classOf[kafka.log.LogManager]) - EasyMock.expect(logManager.getLog(TopicAndPartition(topic, partitionId))).andReturn(Some(log)).anyTimes() - EasyMock.replay(logManager) + // set the replica manager with the partition + val time = new MockTime + val jTime = new JMockTime + val metrics = new Metrics + val leaderLEO = 20L + val followerLEO = 15L + val partitionHW = 5 - 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.expect(replicaManager.readMessageSets(EasyMock.anyObject())).andReturn({ - val fetchInfo = log.read(0, fetchSize, Some(hw)) - val partitionData = new FetchResponsePartitionData(ErrorMapping.NoError, hw.toLong, fetchInfo.messageSet) - Map(TopicAndPartition(topic, partitionId) -> new PartitionDataAndOffset(partitionData, fetchInfo.fetchOffset)) - }).anyTimes() - EasyMock.replay(replicaManager) - - val partition = getPartitionWithAllReplicasInISR(topic, partitionId, time, configs.head.brokerId, log, hw, replicaManager) - partition.getReplica(configs(1).brokerId).get.logEndOffset = new LogOffsetMetadata(leo - 5L, 0L, leo.toInt - 5) - - EasyMock.reset(replicaManager) - EasyMock.expect(replicaManager.config).andReturn(configs.head).anyTimes() - EasyMock.expect(replicaManager.getLeaderReplicaIfLocal(topic, partitionId)).andReturn(partition.leaderReplicaIfLocal().get).anyTimes() - EasyMock.expect(replicaManager.initWithRequestPurgatory(EasyMock.anyObject(), EasyMock.anyObject())) - EasyMock.expect(replicaManager.readMessageSets(EasyMock.anyObject())).andReturn({ - val fetchInfo = log.read(0, fetchSize, Some(hw)) - val partitionData = new FetchResponsePartitionData(ErrorMapping.NoError, hw.toLong, fetchInfo.messageSet) - Map(TopicAndPartition(topic, partitionId) -> new PartitionDataAndOffset(partitionData, fetchInfo.fetchOffset)) - }).anyTimes() - - EasyMock.replay(replicaManager) - - val offsetManager = EasyMock.createMock(classOf[kafka.server.OffsetManager]) - - 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, offsetManager, zkClient, configs.head.brokerId, configs.head, controller) - - val partitionStateInfo = EasyMock.createNiceMock(classOf[PartitionStateInfo]) - apis.metadataCache.addOrUpdatePartitionInfo(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 fetchSize = 100 + val messagesToHW = new Message("messageToHW".getBytes()) + val messagesToLEO = new Message("messageToLEO".getBytes()) - /** - * 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 topic = "test-topic" + val partitionId = 0 + val topicAndPartition = TopicAndPartition(topic, partitionId) - val messages = new Message("test-message".getBytes()) + val fetchInfo = Collections.singletonMap(topicAndPartition, PartitionFetchInfo(0, fetchSize)).toMap - val followerReplicaId = configs(1).brokerId - val followerLEO = 15 + var replicaManager: ReplicaManager = null + @Before + def 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( + // 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(followerLEO, 0L, followerLEO), - new ByteBufferMessageSet(messages) + 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.expect(replicaManager.readMessageSets(EasyMock.anyObject())).andReturn({ - val fetchInfo = log.read(followerLEO, Integer.MAX_VALUE, None) - val partitionData = new FetchResponsePartitionData(ErrorMapping.NoError, hw.toLong, fetchInfo.messageSet) - Map(TopicAndPartition(topic, partitionId) -> new PartitionDataAndOffset(partitionData, fetchInfo.fetchOffset)) - }).anyTimes() - EasyMock.replay(replicaManager) - - val partition = getPartitionWithAllReplicasInISR(topic, partitionId, time, configs.head.brokerId, log, hw, replicaManager) - partition.getReplica(followerReplicaId).get.logEndOffset = new LogOffsetMetadata(followerLEO.asInstanceOf[Long], 0L, followerLEO) - - EasyMock.reset(replicaManager) - EasyMock.expect(replicaManager.config).andReturn(configs.head).anyTimes() - EasyMock.expect(replicaManager.updateReplicaLEOAndPartitionHW(topic, partitionId, followerReplicaId, new LogOffsetMetadata(followerLEO.asInstanceOf[Long], 0L, 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.expect(replicaManager.initWithRequestPurgatory(EasyMock.anyObject(), EasyMock.anyObject())) - EasyMock.expect(replicaManager.readMessageSets(EasyMock.anyObject())).andReturn({ - val fetchInfo = log.read(followerLEO, Integer.MAX_VALUE, None) - val partitionData = new FetchResponsePartitionData(ErrorMapping.NoError, hw.toLong, fetchInfo.messageSet) - Map(TopicAndPartition(topic, partitionId) -> new PartitionDataAndOffset(partitionData, fetchInfo.fetchOffset)) - }).anyTimes() - EasyMock.expect(replicaManager.unblockDelayedProduceRequests(EasyMock.anyObject())).anyTimes() - EasyMock.replay(replicaManager) - - val offsetManager = EasyMock.createMock(classOf[kafka.server.OffsetManager]) - - val controller = EasyMock.createMock(classOf[kafka.controller.KafkaController]) - - val requestChannel = new RequestChannel(2, 5) - val apis = new KafkaApis(requestChannel, replicaManager, offsetManager, zkClient, configs.head.brokerId, configs.head, controller) - val partitionStateInfo = EasyMock.createNiceMock(classOf[PartitionStateInfo]) - apis.metadataCache.addOrUpdatePartitionInfo(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, metrics, time, jTime, zkClient, scheduler, logManager, + new AtomicBoolean(false)) + + // add the partition with two replicas, both in ISR + val partition = replicaManager.getOrCreatePartition(topic, partitionId) - private def getPartitionWithAllReplicasInISR(topic: String, partitionId: Int, time: Time, leaderId: Int, - localLog: Log, leaderHW: Long, replicaManager: ReplicaManager): Partition = { - val partition = new Partition(topic, partitionId, time, replicaManager) - val leaderReplica = new Replica(leaderId, partition, time, 0, Some(localLog)) + // 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) - val allReplicas = getFollowerReplicas(partition, leaderId, time) :+ leaderReplica + // 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)) + + // 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 = new LogOffsetMetadata(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) - } + @After + def tearDown() { + replicaManager.shutdown(false) + metrics.close() } + /** + * 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 + */ + @Test + 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/server/SslReplicaFetchTest.scala b/core/src/test/scala/unit/kafka/server/SslReplicaFetchTest.scala new file mode 100644 index 0000000000000..98580528fb15e --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/SslReplicaFetchTest.scala @@ -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 kafka.server + +import java.io.File + +class SslReplicaFetchTest extends BaseReplicaFetchTest { + protected def trustStoreFile = Some(File.createTempFile("truststore", ".jks")) +} diff --git a/core/src/test/scala/unit/kafka/server/ThrottledResponseExpirationTest.scala b/core/src/test/scala/unit/kafka/server/ThrottledResponseExpirationTest.scala new file mode 100644 index 0000000000000..778f3f8d75aba --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/ThrottledResponseExpirationTest.scala @@ -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 kafka.server + + +import java.util.Collections +import java.util.concurrent.{DelayQueue, TimeUnit} + +import org.apache.kafka.common.metrics.MetricConfig +import org.apache.kafka.common.utils.MockTime +import org.junit.{Assert, Before, Test} + +class ThrottledResponseExpirationTest { + private val time = new MockTime + private var numCallbacks: Int = 0 + private val metrics = new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), + Collections.emptyList(), + time) + + def callback(delayTimeMs: Int) { + numCallbacks += 1 + } + + @Before + def beforeMethod() { + numCallbacks = 0 + } + + @Test + def testExpire() { + val clientMetrics = new ClientQuotaManager(ClientQuotaManagerConfig(), metrics, "producer", time) + + val delayQueue = new DelayQueue[ThrottledResponse]() + val reaper = new clientMetrics.ThrottledRequestReaper(delayQueue) + try { + // Add 4 elements to the queue out of order. Add 2 elements with the same expire timestamp + delayQueue.add(new ThrottledResponse(time, 10, callback)) + delayQueue.add(new ThrottledResponse(time, 30, callback)) + delayQueue.add(new ThrottledResponse(time, 30, callback)) + delayQueue.add(new ThrottledResponse(time, 20, callback)) + + for(itr <- 1 to 3) { + time.sleep(10) + reaper.doWork() + Assert.assertEquals(itr, numCallbacks) + + } + reaper.doWork() + Assert.assertEquals(4, numCallbacks) + Assert.assertEquals(0, delayQueue.size()) + reaper.doWork() + Assert.assertEquals(4, numCallbacks) + } finally { + clientMetrics.shutdown() + } + } + + @Test + def testThrottledRequest() { + val t1: ThrottledResponse = new ThrottledResponse(time, 10, callback) + val t2: ThrottledResponse = new ThrottledResponse(time, 20, callback) + val t3: ThrottledResponse = new ThrottledResponse(time, 20, callback) + Assert.assertEquals(10, t1.throttleTimeMs) + Assert.assertEquals(20, t2.throttleTimeMs) + Assert.assertEquals(20, t3.throttleTimeMs) + + for(itr <- 0 to 2) { + Assert.assertEquals(10 - 10*itr, t1.getDelay(TimeUnit.MILLISECONDS)) + Assert.assertEquals(20 - 10*itr, t2.getDelay(TimeUnit.MILLISECONDS)) + Assert.assertEquals(20 - 10*itr, t3.getDelay(TimeUnit.MILLISECONDS)) + time.sleep(10) + } + } +} diff --git a/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala b/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala new file mode 100644 index 0000000000000..7c08e0938348b --- /dev/null +++ b/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.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 unit.kafka.tools + +import java.io.FileOutputStream + +import kafka.consumer.{BaseConsumer, BaseConsumerRecord} +import kafka.tools.{ConsoleConsumer, MessageFormatter} +import kafka.utils.TestUtils +import org.easymock.EasyMock +import org.junit.Assert._ +import org.junit.Test +import org.scalatest.junit.JUnitSuite + +class ConsoleConsumerTest extends JUnitSuite { + + @Test + def shouldLimitReadsToMaxMessageLimit() { + //Mocks + val consumer = EasyMock.createNiceMock(classOf[BaseConsumer]) + val formatter = EasyMock.createNiceMock(classOf[MessageFormatter]) + + //Stubs + val record = new BaseConsumerRecord("foo", 1, 1, Array[Byte](), Array[Byte]()) + + //Expectations + val messageLimit: Int = 10 + EasyMock.expect(formatter.writeTo(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject())).times(messageLimit) + EasyMock.expect(consumer.receive()).andReturn(record).times(messageLimit) + + EasyMock.replay(consumer) + EasyMock.replay(formatter) + + //Test + ConsoleConsumer.process(messageLimit, formatter, consumer, true) + } + + @Test + def shouldParseValidOldConsumerValidConfig() { + //Given + val args: Array[String] = Array( + "--zookeeper", "localhost:2181", + "--topic", "test", + "--from-beginning") + + //When + val config = new ConsoleConsumer.ConsumerConfig(args) + + //Then + assertFalse(config.useNewConsumer) + assertEquals("localhost:2181", config.zkConnectionStr) + assertEquals("test", config.topicArg) + assertEquals(true, config.fromBeginning) + } + + @Test + def shouldParseValidNewConsumerValidConfig() { + //Given + val args: Array[String] = Array( + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--from-beginning", + "--new-consumer") //new + + //When + val config = new ConsoleConsumer.ConsumerConfig(args) + + //Then + assertTrue(config.useNewConsumer) + assertEquals("localhost:9092", config.bootstrapServer) + assertEquals("test", config.topicArg) + assertEquals(true, config.fromBeginning) + } + + + @Test + def shouldParseConfigsFromFile() { + val propsFile = TestUtils.tempFile() + val propsStream = new FileOutputStream(propsFile) + propsStream.write("consumer.timeout.ms=1000".getBytes()) + propsStream.close() + val args: Array[String] = Array( + "--zookeeper", "localhost:2181", + "--topic", "test", + "--consumer.config", propsFile.getAbsolutePath + ) + + val config = new ConsoleConsumer.ConsumerConfig(args) + + assertEquals("1000", config.consumerProps.getProperty("consumer.timeout.ms")) + } +} 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..4a070bd33228a --- /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 org.junit.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 index e832a0179c721..068526eaedb36 100644 --- a/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala @@ -15,11 +15,10 @@ * limitations under the License. */ -package unit.kafka.utils +package kafka.utils -import junit.framework.Assert._ -import org.junit.{Test, After, Before} -import kafka.utils.CommandLineUtils; +import org.junit.Assert._ +import org.junit.Test class CommandLineUtilsTest { diff --git a/core/src/test/scala/unit/kafka/utils/IteratorTemplateTest.scala b/core/src/test/scala/unit/kafka/utils/IteratorTemplateTest.scala index 46a4e899ef293..56f5905cd1dac 100644 --- a/core/src/test/scala/unit/kafka/utils/IteratorTemplateTest.scala +++ b/core/src/test/scala/unit/kafka/utils/IteratorTemplateTest.scala @@ -16,13 +16,13 @@ */ package kafka.utils -import junit.framework.Assert._ +import org.junit.Assert._ import org.scalatest.Assertions 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/JsonTest.scala b/core/src/test/scala/unit/kafka/utils/JsonTest.scala index 93550e8f24071..6c8ed973ccf36 100644 --- a/core/src/test/scala/unit/kafka/utils/JsonTest.scala +++ b/core/src/test/scala/unit/kafka/utils/JsonTest.scala @@ -16,7 +16,7 @@ */ package kafka.utils -import junit.framework.Assert._ +import org.junit.Assert._ import org.junit.{Test, After, Before} class JsonTest { 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 index 84e08557de5ac..b3835f09006c1 100644 --- a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala @@ -17,17 +17,17 @@ 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.junit.{Before, Test} import org.easymock.EasyMock -class ReplicationUtilsTest extends JUnit3Suite with ZooKeeperTestHarness { +class ReplicationUtilsTest extends ZooKeeperTestHarness { val topic = "my-topic-test" val partitionId = 0 val brokerId = 1 @@ -42,7 +42,9 @@ class ReplicationUtilsTest extends JUnit3Suite with ZooKeeperTestHarness { 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) + @Before override def setUp() { super.setUp() ZkUtils.createPersistentPath(zkClient,topicPath,topicData) @@ -50,7 +52,7 @@ class ReplicationUtilsTest extends JUnit3Suite with ZooKeeperTestHarness { @Test def testUpdateLeaderAndIsr() { - val configs = TestUtils.createBrokerConfigs(1).map(new KafkaConfig(_)) + 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) @@ -67,6 +69,8 @@ class ReplicationUtilsTest extends JUnit3Suite with ZooKeeperTestHarness { EasyMock.expect(replicaManager.zkClient).andReturn(zkClient) EasyMock.replay(replicaManager) + ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.IsrChangeNotificationPath) + val replicas = List(0,1) // regular update @@ -92,4 +96,11 @@ class ReplicationUtilsTest extends JUnit3Suite with ZooKeeperTestHarness { 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..7c131fcdf64fe 100644 --- a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala +++ b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala @@ -16,7 +16,7 @@ */ package kafka.utils -import junit.framework.Assert._ +import org.junit.Assert._ import java.util.concurrent.atomic._ import org.junit.{Test, After, Before} import kafka.utils.TestUtils.retry @@ -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 2dbdd3c2f03ca..4a53e1117b3c2 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -18,34 +18,45 @@ 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 java.security.cert.X509Certificate +import javax.net.ssl.X509TrustManager +import charset.Charset +import kafka.security.auth.{Resource, Authorizer, Acl} +import org.apache.kafka.common.protocol.SecurityProtocol import org.apache.kafka.common.utils.Utils._ -import collection.mutable.Map import collection.mutable.ListBuffer -import org.I0Itec.zkclient.ZkClient +import org.I0Itec.zkclient.{ZkClient, ZkConnection} import kafka.server._ import kafka.producer._ import kafka.message._ import kafka.api._ import kafka.cluster.Broker -import kafka.consumer.ConsumerConfig +import kafka.consumer.{ConsumerTimeoutException, KafkaStream, ConsumerConfig} import kafka.serializer.{StringEncoder, DefaultEncoder, Encoder} import kafka.common.TopicAndPartition import kafka.admin.AdminUtils import kafka.producer.ProducerConfig import kafka.log._ -import junit.framework.AssertionFailedError -import junit.framework.Assert._ +import org.junit.Assert._ import org.apache.kafka.clients.producer.KafkaProducer +import org.apache.kafka.clients.consumer.{ConsumerRebalanceListener, KafkaConsumer} +import org.apache.kafka.common.serialization.ByteArrayDeserializer +import org.apache.kafka.clients.CommonClientConfigs +import org.apache.kafka.common.security.ssl.SSLFactory +import org.apache.kafka.common.config.SSLConfigs +import org.apache.kafka.test.TestSSLUtils + +import scala.collection.Map +import scala.collection.JavaConversions._ /** * Utility functions to help with testing @@ -62,23 +73,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 @@ -90,10 +91,10 @@ object TestUtils extends Logging { Runtime.getRuntime().addShutdownHook(new Thread() { override def run() = { - Utils.rm(f) + CoreUtils.rm(f) } }) - + f } @@ -138,28 +139,46 @@ object TestUtils extends Logging { * Create a test config for the given node id */ def createBrokerConfigs(numConfigs: Int, - enableControlledShutdown: Boolean = true): List[Properties] = { - for((port, node) <- choosePorts(numConfigs).zipWithIndex) - yield createBrokerConfig(node, port, enableControlledShutdown) + zkConnect: String, + enableControlledShutdown: Boolean = true, + enableDeleteTopic: Boolean = false, + enableSSL: Boolean = false, + trustStoreFile: Option[File] = None): Seq[Properties] = { + (0 until numConfigs).map(node => createBrokerConfig(node, zkConnect, enableControlledShutdown, enableDeleteTopic, enableSSL = enableSSL, trustStoreFile = trustStoreFile)) + } + + def getBrokerListStrFromServers(servers: Seq[KafkaServer]): String = { + servers.map(s => formatAddress(s.config.hostName, s.boundPort())).mkString(",") } - def getBrokerListStrFromConfigs(configs: Seq[KafkaConfig]): String = { - configs.map(c => formatAddress(c.hostName, c.port)).mkString(",") + def getSSLBrokerListStrFromServers(servers: Seq[KafkaServer]): String = { + servers.map(s => formatAddress(s.config.hostName, s.boundPort(SecurityProtocol.SSL))).mkString(",") } /** * Create a test config for the given node id */ - def createBrokerConfig(nodeId: Int, port: Int = choosePort(), - enableControlledShutdown: Boolean = true): Properties = { + def createBrokerConfig(nodeId: Int, zkConnect: String, + enableControlledShutdown: Boolean = true, + enableDeleteTopic: Boolean = false, + port: Int = RandomPort, enableSSL: Boolean = false, sslPort: Int = RandomPort, trustStoreFile: Option[File] = None): Properties = { val props = new Properties - props.put("broker.id", nodeId.toString) - props.put("host.name", "localhost") - props.put("port", port.toString) + var listeners: String = "PLAINTEXT://localhost:"+port.toString + if (nodeId >= 0) props.put("broker.id", nodeId.toString) + if (enableSSL) + listeners = listeners + "," + "SSL://localhost:"+sslPort.toString + props.put("listeners", listeners) 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") + if (enableSSL) { + props.putAll(addSSLConfigs(SSLFactory.Mode.SERVER, true, trustStoreFile, "server"+nodeId)) + } + props.put("port", port.toString) props } @@ -168,10 +187,14 @@ object TestUtils extends Logging { * 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]) : scala.collection.immutable.Map[Int, Option[Int]] = { + 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) + 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) @@ -378,19 +401,59 @@ object TestUtils extends Logging { metadataFetchTimeout: Long = 3000L, blockOnBufferFull: Boolean = true, bufferSize: Long = 1024L * 1024L, - retries: Int = 0) : KafkaProducer = { + retries: Int = 0, + lingerMs: Long = 0, + enableSSL: Boolean = false, + trustStoreFile: Option[File] = None) : 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") - return new KafkaProducer(producerProps) + 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") + if (enableSSL) { + producerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SSL") + producerProps.putAll(addSSLConfigs(SSLFactory.Mode.CLIENT, false, trustStoreFile, "producer")) + } + 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, + partitionAssignmentStrategy: String = "blah", + sessionTimeout: Int = 30000, + enableSSL: Boolean = false, + trustStoreFile: Option[File] = None) : 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") + consumerProps.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, partitionAssignmentStrategy) + consumerProps.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, sessionTimeout.toString) + if (enableSSL) { + consumerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SSL") + consumerProps.putAll(addSSLConfigs(SSLFactory.Mode.CLIENT, false, trustStoreFile, "consumer")) + } + new KafkaConsumer[Array[Byte],Array[Byte]](consumerProps) } /** @@ -421,7 +484,7 @@ object TestUtils extends Logging { } 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) } @@ -430,21 +493,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)) + def createBrokersInZk(zkClient: ZkClient, zkConnection: ZkConnection, ids: Seq[Int]): Seq[Broker] = { + val brokers = ids.map(id => new Broker(id, "localhost", 6667, SecurityProtocol.PLAINTEXT)) + brokers.foreach(b => ZkUtils.registerBrokerInZk(zkClient, zkConnection, b.id, "localhost", 6667, b.endPoints, 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 } @@ -479,7 +542,7 @@ object TestUtils extends Logging { 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, @@ -552,7 +615,7 @@ object TestUtils extends Logging { fail("Timing out after %d ms since leader is not elected or changed for partition [%s,%d]" .format(timeoutMs, topic, partition)) - return leader + leader } /** @@ -567,7 +630,7 @@ object TestUtils extends Logging { block return } catch { - case e: AssertionFailedError => + case e: AssertionError => val ellapsed = System.currentTimeMillis - startTime if(ellapsed > maxWaitMs) { throw e @@ -693,33 +756,198 @@ object TestUtils extends Logging { ZkUtils.pathExists(zkClient, ZkUtils.ReassignPartitionsPath) } + def verifyNonDaemonThreadsStatus(threadNamePrefix: String) { + assertEquals(0, Thread.getAllStackTraces.keySet().toArray + .map(_.asInstanceOf[Thread]) + .count(t => !t.isDaemon && t.isAlive && t.getName.startsWith(threadNamePrefix))) + } /** * 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()) = - { - 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 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) + } + + def addSSLConfigs(mode: SSLFactory.Mode, clientCert: Boolean, trustStoreFile: Option[File], certAlias: String): Properties = { + if (!trustStoreFile.isDefined) { + throw new Exception("enableSSL set to true but no trustStoreFile provided") + } + + val sslConfigs = { + if (mode == SSLFactory.Mode.SERVER) { + val sslConfigs = TestSSLUtils.createSSLConfig(true, true, mode, trustStoreFile.get, certAlias) + sslConfigs.put(KafkaConfig.InterBrokerSecurityProtocolProp, SecurityProtocol.SSL.name) + sslConfigs + } + else + TestSSLUtils.createSSLConfig(clientCert, false, mode, trustStoreFile.get, certAlias) + } + + val sslProps = new Properties() + sslConfigs.foreach { case (k, v) => sslProps.put(k, v) } + sslProps + } + + // a X509TrustManager to trust self-signed certs for unit tests. + def trustAllCerts: X509TrustManager = { + val trustManager = new X509TrustManager() { + override def getAcceptedIssuers: Array[X509Certificate] = { + null + } + override def checkClientTrusted(certs: Array[X509Certificate], authType: String) { + } + override def checkServerTrusted(certs: Array[X509Certificate], authType: String) { + } + } + trustManager + } + + def waitAndVerifyAcls(expected: Set[Acl], authorizer: Authorizer, resource: Resource) = { + TestUtils.waitUntilTrue(() => authorizer.getAcls(resource) == expected, + s"expected acls $expected but got ${authorizer.getAcls(resource)}", waitTime = 10000) } -} -object TestZKUtils { - val zookeeperConnect = "127.0.0.1:" + TestUtils.choosePort() } 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 0d0f0e2fba367..9e8869c44a115 --- a/core/src/test/scala/unit/kafka/utils/UtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/UtilsTest.scala @@ -24,8 +24,9 @@ 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 { @@ -34,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()) @@ -48,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()) @@ -75,10 +76,10 @@ class UtilsTest extends JUnitSuite { @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 @@ -87,7 +88,7 @@ class UtilsTest extends JUnitSuite { 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)) } } @@ -95,8 +96,8 @@ class UtilsTest extends JUnitSuite { 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) @@ -104,6 +105,42 @@ class UtilsTest extends JUnitSuite { 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() @@ -114,4 +151,54 @@ 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..a018ddef5f238 --- /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 org.junit.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..95de3780c6f47 --- /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 org.junit.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 3151561508938..2bca2cf5a554d --- a/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala +++ b/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala @@ -21,23 +21,25 @@ import org.apache.zookeeper.server.ZooKeeperServer 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 factory = new NIOServerCnxnFactory() - factory.configure(new InetSocketAddress("127.0.0.1", getPort(connectString)), 0) + 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..2bf658c5f244c 100644 --- a/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala +++ b/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala @@ -18,19 +18,23 @@ package kafka.zk import kafka.consumer.ConsumerConfig -import org.I0Itec.zkclient.ZkClient -import kafka.utils.{ZkUtils, ZKStringSerializer} +import kafka.utils.ZkUtils +import kafka.utils.ZKCheckedEphemeral import kafka.utils.TestUtils -import org.junit.Assert -import org.scalatest.junit.JUnit3Suite +import org.apache.zookeeper.CreateMode +import org.apache.zookeeper.WatchedEvent +import org.apache.zookeeper.Watcher +import org.apache.zookeeper.ZooDefs.Ids +import org.I0Itec.zkclient.exception.{ZkException,ZkNodeExistsException} +import org.junit.{Test, Assert} -class ZKEphemeralTest extends JUnit3Suite with ZooKeeperTestHarness { +class ZKEphemeralTest extends ZooKeeperTestHarness { var zkSessionTimeoutMs = 1000 + @Test 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,9 +46,100 @@ 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) } + + /***** + ***** Tests for ZkWatchedEphemeral + *****/ + + /** + * Tests basic creation + */ + @Test + def testZkWatchedEphemeral = { + val path = "/zwe-test" + testCreation(path) + } + + /** + * Tests recursive creation + */ + @Test + def testZkWatchedEphemeralRecursive = { + val path = "/zwe-test-parent/zwe-test" + testCreation(path) + } + + private def testCreation(path: String) { + val zk = zkConnection.getZookeeper + val zwe = new ZKCheckedEphemeral(path, "", zk) + var created = false + var counter = 10 + + zk.exists(path, new Watcher() { + def process(event: WatchedEvent) { + if(event.getType == Watcher.Event.EventType.NodeCreated) { + created = true + } + } + }) + zwe.create() + // Waits until the znode is created + TestUtils.waitUntilTrue(() => ZkUtils.pathExists(zkClient, path), + "Znode %s wasn't created".format(path)) + } + + /** + * Tests that it fails in the presence of an overlapping + * session. + */ + @Test + def testOverlappingSessions = { + val path = "/zwe-test" + val zk1 = zkConnection.getZookeeper + + //Creates a second session + val (_, zkConnection2) = ZkUtils.createZkClientAndConnection(zkConnect, zkSessionTimeoutMs, zkConnectionTimeout) + val zk2 = zkConnection2.getZookeeper + var zwe = new ZKCheckedEphemeral(path, "", zk2) + + // Creates znode for path in the first session + zk1.create(path, Array[Byte](), Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL) + + //Bootstraps the ZKWatchedEphemeral object + var gotException = false; + try { + zwe.create() + } catch { + case e: ZkNodeExistsException => + gotException = true + } + Assert.assertTrue(gotException) + } + + /** + * Tests if succeeds with znode from the same session + * + */ + @Test + def testSameSession = { + val path = "/zwe-test" + val zk = zkConnection.getZookeeper + // Creates znode for path in the first session + zk.create(path, Array[Byte](), Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL) + + var zwe = new ZKCheckedEphemeral(path, "", zk) + //Bootstraps the ZKWatchedEphemeral object + var gotException = false; + try { + zwe.create() + } catch { + case e: ZkNodeExistsException => + gotException = true + } + Assert.assertFalse(gotException) + } } 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..241eea51d9142 --- /dev/null +++ b/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala @@ -0,0 +1,153 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 kafka.consumer.ConsumerConfig +import kafka.utils.{ZkPath, TestUtils, ZkUtils} +import org.apache.kafka.common.config.ConfigException +import org.junit.Assert._ +import org.junit.Test + +class ZKPathTest extends ZooKeeperTestHarness { + + val path: String = "/some_dir" + val zkSessionTimeoutMs = 1000 + def zkConnectWithInvalidRoot: String = zkConnect + "/ghost" + + @Test + 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") + } + } + + @Test + 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") + } + + assertTrue("Failed to create persistent path", ZkUtils.pathExists(zkClient, path)) + } + + @Test + 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") + } + } + + @Test + 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") + } + + assertTrue("Failed to create persistent path", ZkUtils.pathExists(zkClient, path)) + } + + @Test + 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") + } + } + + @Test + 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") + } + + assertTrue("Failed to create ephemeral path", ZkUtils.pathExists(zkClient, path)) + } + + @Test + 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") + } + } + + @Test + 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") + } + + 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 67d9c4bab270c..3e1c6e0ceecc1 --- a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala +++ b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala @@ -17,27 +17,34 @@ package kafka.zk -import org.scalatest.junit.JUnit3Suite -import org.I0Itec.zkclient.ZkClient -import kafka.utils.{ZKStringSerializer, TestZKUtils, Utils} +import org.I0Itec.zkclient.{ZkClient, ZkConnection} +import kafka.utils.{ZkUtils, CoreUtils} +import org.junit.{After, Before} +import org.scalatest.junit.JUnitSuite -trait ZooKeeperTestHarness extends JUnit3Suite { - val zkConnect: String = TestZKUtils.zookeeperConnect +trait ZooKeeperTestHarness extends JUnitSuite { + var zkPort: Int = -1 var zookeeper: EmbeddedZookeeper = null var zkClient: ZkClient = null + var zkConnection : ZkConnection = null val zkConnectionTimeout = 6000 val zkSessionTimeout = 6000 - override def setUp() { - super.setUp - zookeeper = new EmbeddedZookeeper(zkConnect) - zkClient = new ZkClient(zookeeper.connectString, zkSessionTimeout, zkConnectionTimeout, ZKStringSerializer) + def zkConnect: String = "127.0.0.1:" + zkPort + + @Before + def setUp() { + zookeeper = new EmbeddedZookeeper() + zkPort = zookeeper.port + val (client, connection) = ZkUtils.createZkClientAndConnection(zkConnect, zkSessionTimeout, zkConnectionTimeout) + zkClient = client + zkConnection = connection } - override def tearDown() { - Utils.swallow(zkClient.close()) - Utils.swallow(zookeeper.shutdown()) - super.tearDown + @After + def tearDown() { + CoreUtils.swallow(zkClient.close()) + CoreUtils.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/docs/api.html b/docs/api.html new file mode 100644 index 0000000000000..2b8bad7757eae --- /dev/null +++ b/docs/api.html @@ -0,0 +1,154 @@ + + +We are in the process of rewritting the JVM clients for Kafka. As of 0.8.2 Kafka includes a newly rewritten Java producer. The next release will include an equivalent Java consumer. These new clients are meant to supplant the existing Scala clients, but for compatability they will co-exist for some time. These clients are available in a seperate jar with minimal dependencies, while the old Scala clients remain packaged with the server. + +

                  2.1 Producer API

                  + +As of the 0.8.2 release we encourage all new development to use the new Java producer. This client is production tested and generally both faster and more fully featured than the previous Scala client. You can use this client by adding a dependency on the client jar using the following maven co-ordinates: +
                  +	<dependency>
                  +	    <groupId>org.apache.kafka</groupId>
                  +	    <artifactId>kafka-clients</artifactId>
                  +	    <version>0.8.2.0</version>
                  +	</dependency>
                  +
                  + +Examples showing how to use the producer are given in the +javadocs. + +

                  +For those interested in the legacy Scala producer api, information can be found +here. +

                  + +

                  2.2 High Level Consumer API

                  +
                  +class Consumer {
                  +  /**
                  +   *  Create a ConsumerConnector
                  +   *
                  +   *  @param config  at the minimum, need to specify the groupid of the consumer and the zookeeper
                  +   *                 connection string zookeeper.connect.
                  +   */
                  +  public static kafka.javaapi.consumer.ConsumerConnector createJavaConsumerConnector(ConsumerConfig config);
                  +}
                  +
                  +/**
                  + *  V: type of the message
                  + *  K: type of the optional key assciated with the message
                  + */
                  +public interface kafka.javaapi.consumer.ConsumerConnector {
                  +  /**
                  +   *  Create a list of message streams of type T for each topic.
                  +   *
                  +   *  @param topicCountMap  a map of (topic, #streams) pair
                  +   *  @param decoder a decoder that converts from Message to T
                  +   *  @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.
                  +   */
                  +  public <K,V> Map<String, List<KafkaStream<K,V>>>
                  +    createMessageStreams(Map<String, Integer> topicCountMap, Decoder<K> keyDecoder, Decoder<V> valueDecoder);
                  +
                  +  /**
                  +   *  Create a list of message streams of type T for each topic, using the default decoder.
                  +   */
                  +  public Map<String, List<KafkaStream<byte[], byte[]>>> createMessageStreams(Map<String, Integer> topicCountMap);
                  +
                  +  /**
                  +   *  Create a list of message streams for topics matching a wildcard.
                  +   *
                  +   *  @param topicFilter a TopicFilter that specifies which topics to
                  +   *                    subscribe to (encapsulates a whitelist or a blacklist).
                  +   *  @param numStreams the number of message streams to return.
                  +   *  @param keyDecoder a decoder that decodes the message key
                  +   *  @param valueDecoder a decoder that decodes the message itself
                  +   *  @return a list of KafkaStream. Each stream supports an
                  +   *          iterator over its MessageAndMetadata elements.
                  +   */
                  +  public <K,V> List<KafkaStream<K,V>>
                  +    createMessageStreamsByFilter(TopicFilter topicFilter, int numStreams, Decoder<K> keyDecoder, Decoder<V> valueDecoder);
                  +
                  +  /**
                  +   *  Create a list of message streams for topics matching a wildcard, using the default decoder.
                  +   */
                  +  public List<KafkaStream<byte[], byte[]>> createMessageStreamsByFilter(TopicFilter topicFilter, int numStreams);
                  +
                  +  /**
                  +   *  Create a list of message streams for topics matching a wildcard, using the default decoder, with one stream.
                  +   */
                  +  public List<KafkaStream<byte[], byte[]>> createMessageStreamsByFilter(TopicFilter topicFilter);
                  +
                  +  /**
                  +   *  Commit the offsets of all topic/partitions connected by this connector.
                  +   */
                  +  public void commitOffsets();
                  +
                  +  /**
                  +   *  Shut down the connector
                  +   */
                  +  public void shutdown();
                  +}
                  +
                  +
                  +You can follow +this example to learn how to use the high level consumer api. +

                  2.3 Simple Consumer API

                  +
                  +class kafka.javaapi.consumer.SimpleConsumer {
                  +  /**
                  +   *  Fetch a set of messages from a topic.
                  +   *
                  +   *  @param request specifies the topic name, topic partition, starting byte offset, maximum bytes to be fetched.
                  +   *  @return a set of fetched messages
                  +   */
                  +  public FetchResponse fetch(kafka.javaapi.FetchRequest request);
                  +
                  +  /**
                  +   *  Fetch metadata for a sequence of topics.
                  +   *
                  +   *  @param request specifies the versionId, clientId, sequence of topics.
                  +   *  @return metadata for each topic in the request.
                  +   */
                  +  public kafka.javaapi.TopicMetadataResponse send(kafka.javaapi.TopicMetadataRequest request);
                  +
                  +  /**
                  +   *  Get a list of valid offsets (up to maxSize) before the given time.
                  +   *
                  +   *  @param request a [[kafka.javaapi.OffsetRequest]] object.
                  +   *  @return a [[kafka.javaapi.OffsetResponse]] object.
                  +   */
                  +  public kafka.javaapi.OffsetResponse getOffsetsBefore(OffsetRequest request);
                  +
                  +  /**
                  +   * Close the SimpleConsumer.
                  +   */
                  +  public void close();
                  +}
                  +
                  +For most applications, the high level consumer Api is good enough. Some applications want features not exposed to the high level consumer yet (e.g., set initial offset when restarting the consumer). They can instead use our low level SimpleConsumer Api. The logic will be a bit more complicated and you can follow the example in +here. + +

                  2.4 Kafka Hadoop Consumer API

                  +

                  +Providing a horizontally scalable solution for aggregating and loading data into Hadoop was one of our basic use cases. To support this use case, we provide a Hadoop-based consumer which spawns off many map tasks to pull data from the Kafka cluster in parallel. This provides extremely fast pull-based Hadoop data load capabilities (we were able to fully saturate the network with only a handful of Kafka servers). +

                  + +

                  +Usage information on the hadoop consumer can be found here. +

                  diff --git a/docs/configuration.html b/docs/configuration.html new file mode 100644 index 0000000000000..c3cc13e393124 --- /dev/null +++ b/docs/configuration.html @@ -0,0 +1,928 @@ + + +Kafka uses key-value pairs in the property file format for configuration. These values can be supplied either from a file or programmatically. + +

                  3.1 Broker Configs

                  + +The essential configurations are the following: +
                    +
                  • broker.id +
                  • log.dirs +
                  • zookeeper.connect +
                  + +Topic-level configurations and defaults are discussed in more detail below. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
                  PropertyDefaultDescription
                  broker.idEach broker is uniquely identified by a non-negative integer id. This id serves as the broker's "name" and allows the broker to be moved to a different host/port without confusing consumers. You can choose any number you like so long as it is unique. +
                  log.dirs/tmp/kafka-logsA comma-separated list of one or more directories in which Kafka data is stored. Each new partition that is created will be placed in the directory which currently has the fewest partitions.
                  port9092The port on which the server accepts client connections.
                  zookeeper.connectnullSpecifies the ZooKeeper connection string in the form hostname:port, where hostname and port are the host and port for a node in your ZooKeeper cluster. To allow connecting through other ZooKeeper nodes when that host is down you can also specify multiple hosts in the form hostname1:port1,hostname2:port2,hostname3:port3. +

                  +ZooKeeper also allows you to add a "chroot" path which will make all kafka data for this cluster appear under a particular path. This is a way to setup multiple Kafka clusters or other applications on the same ZooKeeper cluster. To do this give a connection string in the form hostname1:port1,hostname2:port2,hostname3:port3/chroot/path which would put all this cluster's data under the path /chroot/path. Note that consumers must use the same connection string.

                  message.max.bytes1000000The maximum size of a message that the server can receive. It is important that this property be in sync with the maximum fetch size your consumers use or else an unruly producer will be able to publish messages too large for consumers to consume.
                  num.network.threads3The number of network threads that the server uses for handling network requests. You probably don't need to change this.
                  num.io.threads8The number of I/O threads that the server uses for executing requests. You should have at least as many threads as you have disks.
                  background.threads10The number of threads to use for various background processing tasks such as file deletion. You should not need to change this.
                  queued.max.requests500The number of requests that can be queued up for processing by the I/O threads before the network threads stop reading in new requests.
                  host.namenull +

                  Hostname of broker. If this is set, it will only bind to this address. If this is not set, it will bind to all interfaces, and publish one to ZK.

                  +
                  advertised.host.namenull +

                  If this is set this is the hostname that will be given out to producers, consumers, and other brokers to connect to.

                  +
                  advertised.portnull +

                  The port to give out to producers, consumers, and other brokers to use in establishing connections. This only needs to be set if this port is different from the port the server should bind to.

                  +
                  socket.send.buffer.bytes100 * 1024The SO_SNDBUFF buffer the server prefers for socket connections.
                  socket.receive.buffer.bytes100 * 1024The SO_RCVBUFF buffer the server prefers for socket connections.
                  socket.request.max.bytes100 * 1024 * 1024The maximum request size the server will allow. This prevents the server from running out of memory and should be smaller than the Java heap size.
                  num.partitions1The default number of partitions per topic if a partition count isn't given at topic creation time.
                  log.segment.bytes1024 * 1024 * 1024The log for a topic partition is stored as a directory of segment files. This setting controls the size to which a segment file will grow before a new segment is rolled over in the log. This setting can be overridden on a per-topic basis (see the per-topic configuration section).
                  log.roll.{ms,hours}24 * 7 hoursThis setting will force Kafka to roll a new log segment even if the log.segment.bytes size has not been reached. This setting can be overridden on a per-topic basis (see the per-topic configuration section).
                  log.cleanup.policydeleteThis can take either the value delete or compact. If delete is set, log segments will be deleted when they reach the size or time limits set. If compact is set log compaction will be used to clean out obsolete records. This setting can be overridden on a per-topic basis (see the per-topic configuration section).
                  log.retention.{ms,minutes,hours}7 daysThe amount of time to keep a log segment before it is deleted, i.e. the default data retention window for all topics. Note that if both log.retention.minutes and log.retention.bytes are both set we delete a segment when either limit is exceeded. This setting can be overridden on a per-topic basis (see the per-topic configuration section).
                  log.retention.bytes-1The amount of data to retain in the log for each topic-partitions. Note that this is the limit per-partition so multiply by the number of partitions to get the total data retained for the topic. Also note that if both log.retention.hours and log.retention.bytes are both set we delete a segment when either limit is exceeded. This setting can be overridden on a per-topic basis (see the per-topic configuration section).
                  log.retention.check.interval.ms5 minutesThe period with which we check whether any log segment is eligible for deletion to meet the retention policies.
                  log.cleaner.enablefalseThis configuration must be set to true for log compaction to run.
                  log.cleaner.threads1The number of threads to use for cleaning logs in log compaction.
                  log.cleaner.io.max.bytes.per.secondDouble.MaxValueThe maximum amount of I/O the log cleaner can do while performing log compaction. This setting allows setting a limit for the cleaner to avoid impacting live request serving.
                  log.cleaner.dedupe.buffer.size500*1024*1024The size of the buffer the log cleaner uses for indexing and deduplicating logs during cleaning. Larger is better provided you have sufficient memory.
                  log.cleaner.io.buffer.size512*1024The size of the I/O chunk used during log cleaning. You probably don't need to change this.
                  log.cleaner.io.buffer.load.factor0.9The load factor of the hash table used in log cleaning. You probably don't need to change this.
                  log.cleaner.backoff.ms15000The interval between checks to see if any logs need cleaning.
                  log.cleaner.min.cleanable.ratio0.5This configuration controls how frequently the log compactor will attempt to clean the log (assuming log compaction is enabled). By default we will avoid cleaning a log where more than 50% of the log has been compacted. This ratio bounds the maximum space wasted in the log by duplicates (at 50% at most 50% of the log could be duplicates). A higher ratio will mean fewer, more efficient cleanings but will mean more wasted space in the log. This setting can be overridden on a per-topic basis (see the per-topic configuration section).
                  log.cleaner.delete.retention.ms1 dayThe amount of time to retain delete tombstone markers for log compacted topics. This setting also gives a bound on the time in which a consumer must complete a read if they begin from offset 0 to ensure that they get a valid snapshot of the final stage (otherwise delete tombstones may be collected before they complete their scan). This setting can be overridden on a per-topic basis (see the per-topic configuration section).
                  log.index.size.max.bytes10 * 1024 * 1024The maximum size in bytes we allow for the offset index for each log segment. Note that we will always pre-allocate a sparse file with this much space and shrink it down when the log rolls. If the index fills up we will roll a new log segment even if we haven't reached the log.segment.bytes limit. This setting can be overridden on a per-topic basis (see the per-topic configuration section).
                  log.index.interval.bytes4096The byte interval at which we add an entry to the offset index. When executing a fetch request the server must do a linear scan for up to this many bytes to find the correct position in the log to begin and end the fetch. So setting this value to be larger will mean larger index files (and a bit more memory usage) but less scanning. However the server will never add more than one index entry per log append (even if more than log.index.interval worth of messages are appended). In general you probably don't need to mess with this value.
                  log.flush.interval.messagesLong.MaxValueThe number of messages written to a log partition before we force an fsync on the log. Setting this lower will sync data to disk more often but will have a major impact on performance. We generally recommend that people make use of replication for durability rather than depending on single-server fsync, however this setting can be used to be extra certain.
                  log.flush.scheduler.interval.msLong.MaxValueThe frequency in ms that the log flusher checks whether any log is eligible to be flushed to disk.
                  log.flush.interval.msLong.MaxValueThe maximum time between fsync calls on the log. If used in conjuction with log.flush.interval.messages the log will be flushed when either criteria is met.
                  log.delete.delay.ms60000The period of time we hold log files around after they are removed from the in-memory segment index. This period of time allows any in-progress reads to complete uninterrupted without locking. You generally don't need to change this.
                  log.flush.offset.checkpoint.interval.ms60000The frequency with which we checkpoint the last flush point for logs for recovery. You should not need to change this.
                  log.segment.delete.delay.ms60000the amount of time to wait before deleting a file from the filesystem.
                  auto.create.topics.enabletrueEnable auto creation of topic on the server. If this is set to true then attempts to produce data or fetch metadata for a non-existent topic will automatically create it with the default replication factor and number of partitions.
                  controller.socket.timeout.ms30000The socket timeout for commands from the partition management controller to the replicas.
                  controller.message.queue.sizeInt.MaxValueThe buffer size for controller-to-broker-channels
                  default.replication.factor1The default replication factor for automatically created topics.
                  replica.lag.time.max.ms10000If a follower hasn't sent any fetch requests for this window of time, the leader will remove the follower from ISR (in-sync replicas) and treat it as dead.
                  replica.socket.timeout.ms30 * 1000The socket timeout for network requests to the leader for replicating data.
                  replica.socket.receive.buffer.bytes64 * 1024The socket receive buffer for network requests to the leader for replicating data.
                  replica.fetch.max.bytes1024 * 1024The number of byes of messages to attempt to fetch for each partition in the fetch requests the replicas send to the leader.
                  replica.fetch.wait.max.ms500The maximum amount of time to wait time for data to arrive on the leader in the fetch requests sent by the replicas to the leader.
                  replica.fetch.min.bytes1Minimum bytes expected for each fetch response for the fetch requests from the replica to the leader. If not enough bytes, wait up to replica.fetch.wait.max.ms for this many bytes to arrive.
                  num.replica.fetchers1 +

                  Number of threads used to replicate messages from leaders. Increasing this value can increase the degree of I/O parallelism in the follower broker.

                  +
                  replica.high.watermark.checkpoint.interval.ms5000The frequency with which each replica saves its high watermark to disk to handle recovery.
                  fetch.purgatory.purge.interval.requests1000The purge interval (in number of requests) of the fetch request purgatory.
                  producer.purgatory.purge.interval.requests1000The purge interval (in number of requests) of the producer request purgatory.
                  zookeeper.session.timeout.ms6000ZooKeeper session timeout. If the server fails to heartbeat to ZooKeeper within this period of time it is considered dead. If you set this too low the server may be falsely considered dead; if you set it too high it may take too long to recognize a truly dead server.
                  zookeeper.connection.timeout.ms6000The maximum amount of time that the client waits to establish a connection to zookeeper.
                  zookeeper.sync.time.ms2000How far a ZK follower can be behind a ZK leader.
                  controlled.shutdown.enabletrueEnable controlled shutdown of the broker. If enabled, the broker will move all leaders on it to some other brokers before shutting itself down. This reduces the unavailability window during shutdown.
                  controlled.shutdown.max.retries3Number of retries to complete the controlled shutdown successfully before executing an unclean shutdown.
                  controlled.shutdown.retry.backoff.ms5000Backoff time between shutdown retries.
                  auto.leader.rebalance.enabletrueIf this is enabled the controller will automatically try to balance leadership for partitions among the brokers by periodically returning leadership to the "preferred" replica for each partition if it is available.
                  leader.imbalance.per.broker.percentage10The percentage of leader imbalance allowed per broker. The controller will rebalance leadership if this ratio goes above + the configured value per broker.
                  leader.imbalance.check.interval.seconds300The frequency with which to check for leader imbalance.
                  offset.metadata.max.bytes4096The maximum amount of metadata to allow clients to save with their offsets.
                  max.connections.per.ipInt.MaxValueThe maximum number of connections that a broker allows from each ip address.
                  max.connections.per.ip.overridesPer-ip or hostname overrides to the default maximum number of connections.
                  connections.max.idle.ms600000Idle connections timeout: the server socket processor threads close the connections that idle more than this.
                  log.roll.jitter.{ms,hours}0The maximum jitter to subtract from logRollTimeMillis.
                  num.recovery.threads.per.data.dir1The number of threads per data directory to be used for log recovery at startup and flushing at shutdown.
                  unclean.leader.election.enabletrueIndicates 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.
                  delete.topic.enablefalseEnable delete topic.
                  offsets.topic.num.partitions50The number of partitions for the offset commit topic. Since changing this after deployment is currently unsupported, we recommend using a higher setting for production (e.g., 100-200).
                  offsets.topic.retention.minutes1440Offsets that are older than this age will be marked for deletion. The actual purge will occur when the log cleaner compacts the offsets topic.
                  offsets.retention.check.interval.ms600000The frequency at which the offset manager checks for stale offsets.
                  offsets.topic.replication.factor3The replication factor for the offset commit topic. A higher setting (e.g., three or four) is recommended in order to ensure higher availability. If the offsets topic is created when fewer brokers than the replication factor then the offsets topic will be created with fewer replicas.
                  offsets.topic.segment.bytes104857600Segment size for the offsets topic. Since it uses a compacted topic, this should be kept relatively low in order to facilitate faster log compaction and loads.
                  offsets.load.buffer.size5242880An offset load occurs when a broker becomes the offset manager for a set of consumer groups (i.e., when it becomes a leader for an offsets topic partition). This setting corresponds to the batch size (in bytes) to use when reading from the offsets segments when loading offsets into the offset manager's cache.
                  offsets.commit.required.acks-1The number of acknowledgements that are required before the offset commit can be accepted. This is similar to the producer's acknowledgement setting. In general, the default should not be overridden.
                  offsets.commit.timeout.ms5000The offset commit will be delayed until this timeout or the required number of replicas have received the offset commit. This is similar to the producer request timeout.
                  inter.broker.protocol.version0.8.3Version of the protocol brokers will use to communicate with each other. This will default for the current version of the broker, but may need to be set to older versions during a rolling upgrade process. In that scenario, upgraded brokers will use the older version of the protocol and therefore will be able to communicate with brokers that were not yet upgraded. See upgrade section for more details.
                  + +

                  More details about broker configuration can be found in the scala class kafka.server.KafkaConfig.

                  + +

                  Topic-level configuration

                  + +Configurations pertinent to topics have both a global default as well an optional per-topic override. If no per-topic configuration is given the global default is used. The override can be set at topic creation time by giving one or more --config options. This example creates a topic named my-topic with a custom max message size and flush rate: +
                  + > bin/kafka-topics.sh --zookeeper localhost:2181 --create --topic my-topic --partitions 1
                  +        --replication-factor 1 --config max.message.bytes=64000 --config flush.messages=1
                  +
                  +Overrides can also be changed or set later using the alter topic command. This example updates the max message size for my-topic: +
                  + > bin/kafka-topics.sh --zookeeper localhost:2181 --alter --topic my-topic
                  +    --config max.message.bytes=128000
                  +
                  + +To remove an override you can do +
                  + > bin/kafka-topics.sh --zookeeper localhost:2181 --alter --topic my-topic
                  +    --deleteConfig max.message.bytes
                  +
                  + +The following are the topic-level configurations. The server's default configuration for this property is given under the Server Default Property heading, setting this default in the server config allows you to change the default given to topics that have no override specified. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
                  PropertyDefaultServer Default PropertyDescription
                  cleanup.policydeletelog.cleanup.policyA string that is either "delete" or "compact". This string designates the retention policy to use on old log segments. The default policy ("delete") will discard old segments when their retention time or size limit has been reached. The "compact" setting will enable log compaction on the topic.
                  delete.retention.ms86400000 (24 hours)log.cleaner.delete.retention.msThe amount of time to retain delete tombstone markers for log compacted topics. This setting also gives a bound on the time in which a consumer must complete a read if they begin from offset 0 to ensure that they get a valid snapshot of the final stage (otherwise delete tombstones may be collected before they complete their scan).
                  flush.messagesNonelog.flush.interval.messagesThis setting allows specifying an interval at which we will force an fsync of data written to the log. For example if this was set to 1 we would fsync after every message; if it were 5 we would fsync after every five messages. In general we recommend you not set this and use replication for durability and allow the operating system's background flush capabilities as it is more efficient. This setting can be overridden on a per-topic basis (see the per-topic configuration section).
                  flush.msNonelog.flush.interval.msThis setting allows specifying a time interval at which we will force an fsync of data written to the log. For example if this was set to 1000 we would fsync after 1000 ms had passed. In general we recommend you not set this and use replication for durability and allow the operating system's background flush capabilities as it is more efficient.
                  index.interval.bytes4096log.index.interval.bytesThis setting controls how frequently Kafka adds an index entry to it's offset index. The default setting ensures that we index a message roughly every 4096 bytes. More indexing allows reads to jump closer to the exact position in the log but makes the index larger. You probably don't need to change this.
                  max.message.bytes1,000,000message.max.bytesThis is largest message size Kafka will allow to be appended to this topic. Note that if you increase this size you must also increase your consumer's fetch size so they can fetch messages this large.
                  min.cleanable.dirty.ratio0.5log.cleaner.min.cleanable.ratioThis configuration controls how frequently the log compactor will attempt to clean the log (assuming log compaction is enabled). By default we will avoid cleaning a log where more than 50% of the log has been compacted. This ratio bounds the maximum space wasted in the log by duplicates (at 50% at most 50% of the log could be duplicates). A higher ratio will mean fewer, more efficient cleanings but will mean more wasted space in the log.
                  min.insync.replicas1min.insync.replicasWhen a producer sets request.required.acks to -1, min.insync.replicas specifies the minimum number of replicas that must acknowledge a write for the write to be considered successful. If this minimum cannot be met, then the producer will raise an exception (either NotEnoughReplicas or NotEnoughReplicasAfterAppend).
                  + When used together, min.insync.replicas and request.required.acks allow you to enforce greater durability guarantees. A typical scenario would be to create a topic with a replication factor of 3, set min.insync.replicas to 2, and produce with request.required.acks of -1. This will ensure that the producer raises an exception if a majority of replicas do not receive a write.
                  retention.bytesNonelog.retention.bytesThis configuration controls the maximum size a log can grow to before we will discard old log segments to free up space if we are using the "delete" retention policy. By default there is no size limit only a time limit.
                  retention.ms7 dayslog.retention.minutesThis configuration controls the maximum time we will retain a log before we will discard old log segments to free up space if we are using the "delete" retention policy. This represents an SLA on how soon consumers must read their data.
                  segment.bytes1 GBlog.segment.bytesThis configuration controls the segment file size for the log. Retention and cleaning is always done a file at a time so a larger segment size means fewer files but less granular control over retention.
                  segment.index.bytes10 MBlog.index.size.max.bytesThis configuration controls the size of the index that maps offsets to file positions. We preallocate this index file and shrink it only after log rolls. You generally should not need to change this setting.
                  segment.ms7 dayslog.roll.hoursThis configuration controls the period of time after which Kafka will force the log to roll even if the segment file isn't full to ensure that retention can delete or compact old data.
                  segment.jitter.ms0log.roll.jitter.{ms,hours}The maximum jitter to subtract from logRollTimeMillis.
                  + +

                  3.2 Consumer Configs

                  +The essential consumer configurations are the following: +
                    +
                  • group.id +
                  • zookeeper.connect +
                  + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
                  PropertyDefaultDescription
                  group.idA string that uniquely identifies the group of consumer processes to which this consumer belongs. By setting the same group id multiple processes indicate that they are all part of the same consumer group.
                  zookeeper.connectSpecifies the ZooKeeper connection string in the form hostname:port where host and port are the host and port of a ZooKeeper server. To allow connecting through other ZooKeeper nodes when that ZooKeeper machine is down you can also specify multiple hosts in the form hostname1:port1,hostname2:port2,hostname3:port3. +

                  + The server may also have a ZooKeeper chroot path as part of it's ZooKeeper connection string which puts its data under some path in the global ZooKeeper namespace. If so the consumer should use the same chroot path in its connection string. For example to give a chroot path of /chroot/path you would give the connection string as hostname1:port1,hostname2:port2,hostname3:port3/chroot/path.

                  consumer.idnull +

                  Generated automatically if not set.

                  +
                  socket.timeout.ms30 * 1000The socket timeout for network requests. The actual timeout set will be max.fetch.wait + socket.timeout.ms.
                  socket.receive.buffer.bytes64 * 1024The socket receive buffer for network requests
                  fetch.message.max.bytes1024 * 1024The number of byes of messages to attempt to fetch for each topic-partition in each fetch request. These bytes will be read into memory for each partition, so this helps control the memory used by the consumer. The fetch request 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.
                  num.consumer.fetchers1The number fetcher threads used to fetch data.
                  auto.commit.enabletrueIf true, periodically commit to ZooKeeper the offset of messages already fetched by the consumer. This committed offset will be used when the process fails as the position from which the new consumer will begin.
                  auto.commit.interval.ms60 * 1000The frequency in ms that the consumer offsets are committed to zookeeper.
                  queued.max.message.chunks2Max number of message chunks buffered for consumption. Each chunk can be up to fetch.message.max.bytes.
                  rebalance.max.retries4When a new consumer joins a consumer group the set of consumers attempt to "rebalance" the load to assign partitions to each consumer. If the set of consumers changes while this assignment is taking place the rebalance will fail and retry. This setting controls the maximum number of attempts before giving up.
                  fetch.min.bytes1The 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.
                  fetch.wait.max.ms100The 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
                  rebalance.backoff.ms2000Backoff time between retries during rebalance.
                  refresh.leader.backoff.ms200Backoff time to wait before trying to determine the leader of a partition that has just lost its leader.
                  auto.offset.resetlargest +

                  What to do when there is no initial offset in ZooKeeper or 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
                  * anything else: throw exception to the consumer

                  +
                  consumer.timeout.ms-1Throw a timeout exception to the consumer if no message is available for consumption after the specified interval
                  exclude.internal.topicstrueWhether messages from internal topics (such as offsets) should be exposed to the consumer.
                  partition.assignment.strategyrangeSelect a strategy for assigning partitions to consumer streams. Possible values: range, roundrobin.
                  client.idgroup id valueThe client id is a user-specified string sent in each request to help trace calls. It should logically identify the application making the request.
                  zookeeper.session.timeout.ms 6000ZooKeeper session timeout. If the consumer fails to heartbeat to ZooKeeper for this period of time it is considered dead and a rebalance will occur.
                  zookeeper.connection.timeout.ms6000The max time that the client waits while establishing a connection to zookeeper.
                  zookeeper.sync.time.ms 2000How far a ZK follower can be behind a ZK leader
                  offsets.storagezookeeperSelect where offsets should be stored (zookeeper or kafka).
                  offsets.channel.backoff.ms1000The backoff period when reconnecting the offsets channel or retrying failed offset fetch/commit requests.
                  offsets.channel.socket.timeout.ms10000Socket timeout when reading responses for offset fetch/commit requests. This timeout is also used for ConsumerMetadata requests that are used to query for the offset manager.
                  offsets.commit.max.retries5Retry 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 originating 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 will be retried and that retry does not count toward this limit.
                  dual.commit.enabledtrueIf 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 version that commits offsets to the broker (instead of directly to ZooKeeper).
                  partition.assignment.strategyrange

                  Select between the "range" or "roundrobin" strategy for assigning partitions to consumer streams.

                  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.) Round-robin assignment is permitted 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.

                  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.

                  + + +

                  More details about consumer configuration can be found in the scala class kafka.consumer.ConsumerConfig.

                  +

                  3.3 Producer Configs

                  +Essential configuration properties for the producer include: +
                    +
                  • metadata.broker.list +
                  • request.required.acks +
                  • producer.type +
                  • serializer.class +
                  + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
                  PropertyDefaultDescription
                  metadata.broker.list +

                  This is for bootstrapping and the producer will only use it for getting metadata (topics, partitions and replicas). The socket connections for sending the actual data will be established based on the broker information returned in the metadata. The format is host1:port1,host2:port2, and the list can be a subset of brokers or a VIP pointing to a subset of brokers.

                  +
                  request.required.acks0 +

                  This value controls when a produce request is considered completed. Specifically, how many other brokers must have committed the data to their log and acknowledged this to the leader? Typical values are +

                    +
                  • 0, which means that the producer never waits for an acknowledgement from the broker (the same behavior as 0.7). This option provides the lowest latency but the weakest durability guarantees (some data will be lost when a server fails). +
                  • 1, which means that the producer gets an acknowledgement after the leader replica has received the data. This option provides better durability as the client waits until the server acknowledges the request as successful (only messages that were written to the now-dead leader but not yet replicated will be lost). +
                  • -1, The producer gets an acknowledgement after all in-sync replicas have received the data. This option provides the greatest level of durability. However, it does not completely eliminate the risk of message loss because the number of in sync replicas may, in rare cases, shrink to 1. If you want to ensure that some minimum number of replicas (typically a majority) receive a write, then you must set the topic-level min.insync.replicas setting. Please read the Replication section of the design documentation for a more in-depth discussion. +
                  +

                  +
                  request.timeout.ms10000The amount of time the broker will wait trying to meet the request.required.acks requirement before sending back an error to the client.
                  producer.typesync +

                  This parameter specifies whether the messages are sent asynchronously in a background thread. Valid values are (1) async for asynchronous send and (2) sync for synchronous send. By setting the producer to async we allow batching together of requests (which is great for throughput) but open the possibility of a failure of the client machine dropping unsent data.

                  +
                  serializer.classkafka.serializer.DefaultEncoderThe serializer class for messages. The default encoder takes a byte[] and returns the same byte[].
                  key.serializer.classThe serializer class for keys (defaults to the same as for messages if nothing is given).
                  partitioner.classkafka.producer.DefaultPartitionerThe partitioner class for partitioning messages amongst sub-topics. The default partitioner is based on the hash of the key.
                  compression.codecnone +

                  This parameter allows you to specify the compression codec for all data generated by this producer. Valid values are "none", "gzip" and "snappy".

                  +
                  compressed.topicsnull +

                  This parameter allows you to set whether compression should be turned on for particular 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

                  +
                  message.send.max.retries3 +

                  This property will cause the producer to automatically retry a failed send request. This property specifies the number of retries when such failures occur. Note that setting a non-zero value here can lead to duplicates in the case of network errors that cause a message to be sent but the acknowledgement to be lost.

                  +
                  retry.backoff.ms100 +

                  Before each retry, the producer refreshes the metadata of relevant topics to see if a new leader has been elected. Since leader election takes a bit of time, this property specifies the amount of time that the producer waits before refreshing the metadata.

                  +
                  topic.metadata.refresh.interval.ms600 * 1000 +

                  The producer generally refreshes the topic metadata from brokers when there is a failure (partition missing, leader not available...). It will also poll regularly (default: every 10min so 600000ms). If you set this to a negative value, metadata will only get refreshed on failure. If you set this to zero, the metadata will get refreshed after each message sent (not recommended). Important note: the refresh happen only AFTER the message is sent, so if the producer never sends a message the metadata is never refreshed

                  +
                  queue.buffering.max.ms5000Maximum time to buffer data when using async mode. For example a setting of 100 will try to batch together 100ms of messages to send at once. This will improve throughput but adds message delivery latency due to the buffering.
                  queue.buffering.max.messages10000The maximum number of unsent messages that can be queued up the producer when using async mode before either the producer must be blocked or data must be dropped.
                  queue.enqueue.timeout.ms-1 +

                  The amount of time to block before dropping messages when running in async mode and the buffer has reached queue.buffering.max.messages. If set to 0 events will be enqueued immediately or dropped if the queue is full (the producer send call will never block). If set to -1 the producer will block indefinitely and never willingly drop a send.

                  +
                  batch.num.messages200The number of messages to send in one batch when using async mode. The producer will wait until either this number of messages are ready to send or queue.buffer.max.ms is reached.
                  send.buffer.bytes100 * 1024Socket write buffer size
                  client.id""The client id is a user-specified string sent in each request to help trace calls. It should logically identify the application making the request.
                  +

                  More details about producer configuration can be found in the scala class kafka.producer.ProducerConfig.

                  + +

                  3.4 New Producer Configs

                  + +We are working on a replacement for our existing producer. The code is available in trunk now and can be considered beta quality. Below is the configuration for the new producer. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
                  NameTypeDefaultImportanceDescription
                  bootstrap.serverslisthighA list of host/port pairs to use for establishing the initial connection to the Kafka cluster. Data will be load balanced over 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). If no server in this list is available sending data will fail until on becomes available.
                  acksstring1highThe 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.
                  • Other settings such as acks=2 are also possible, and will require the given number of acknowledgements but this is generally less useful.
                  buffer.memorylong33554432highThe 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.

                  compression.typestringnonehighThe compression type for all data generated by the producer. The default is none (i.e. no compression). Valid values are none, gzip, or snappy. Compression is of full batches of data, so the efficacy of batching will also impact the compression ratio (more batching means better compression).
                  retriesint0highSetting 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.
                  key.serializerstringhighThe serializer class for keys. The provided class must implement org.apache.kafka.common.serialization.Serializer. Default implementations are available in org.apache.kafka.common.serialization.
                  value.serializerstringhighThe serializer class for messages. The provided class must implement org.apache.kafka.common.serialization.Serializer. Default implementations are available in org.apache.kafka.common.serialization.
                  batch.sizeint16384mediumThe 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.

                  client.idstringmediumThe 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 with the request. The application can set any string it wants as this has no functional purpose other than in logging and metrics.
                  linger.mslong0mediumThe 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.
                  max.request.sizeint1048576mediumThe 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.
                  receive.buffer.bytesint32768mediumThe size of the TCP receive buffer to use when reading data
                  send.buffer.bytesint131072mediumThe size of the TCP send buffer to use when sending data
                  timeout.msint30000mediumThe 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.
                  block.on.buffer.fullbooleantruelowWhen 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.
                  metadata.fetch.timeout.mslong60000lowThe 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 configuration controls the maximum amount of time we will block waiting for the metadata fetch to succeed before throwing an exception back to the client.
                  metadata.max.age.mslong300000lowThe 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.
                  metric.reporterslist[]lowA 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.
                  metrics.num.samplesint2lowThe number of samples maintained to compute metrics.
                  metrics.sample.window.mslong30000lowThe metrics system maintains a configurable number of samples over a fixed window size. This configuration controls the size of the window. For example we might maintain two samples each measured over a 30 second period. When a window expires we erase and overwrite the oldest window.
                  reconnect.backoff.mslong10lowThe amount of time to wait before attempting to reconnect to a given host when a connection fails. This avoids a scenario where the client repeatedly attempts to connect to a host in a tight loop.
                  retry.backoff.mslong100lowThe amount of time to wait before attempting to retry a failed produce request to a given topic partition. This avoids repeated sending-and-failing in a tight loop.
                  max.in.flight.requests.per.connectionint5lowThe number of "in-flight requests" on a given connection is the number of requests currently awaiting acknowledgement on that connection. When the number of in-flight requests is equal to this number, the client will wait for one of them to be acknowledged before sending the next request to that node. It's important to note that one request corresponds to a batch of messages, so increasing this number too much can cause suboptimal batching behavior in the client. This is because the client will not wait long enough to let messages batch together into large batches, and will instead send many small requests which are much less efficient to process. You can explicitly instruct the client to wait a certain amount of time until more messages are available to batch by setting linger.ms. In addition, setting this number high increase throughput but can cause out-of-order message arrivals.
                  diff --git a/docs/design.html b/docs/design.html new file mode 100644 index 0000000000000..b1e4387373381 --- /dev/null +++ b/docs/design.html @@ -0,0 +1,355 @@ + + +

                  4.1 Motivation

                  +

                  +We designed Kafka to be able to act as a unified platform for handling all the real-time data feeds a large company might have. To do this we had to think through a fairly broad set of use cases. +

                  +It would have to have high-throughput to support high volume event streams such as real-time log aggregation. +

                  +It would need to deal gracefully with large data backlogs to be able to support periodic data loads from offline systems. +

                  +It also meant the system would have to handle low-latency delivery to handle more traditional messaging use-cases. +

                  +We wanted to support partitioned, distributed, real-time processing of these feeds to create new, derived feeds. This motivated our partitioning and consumer model. +

                  +Finally in cases where the stream is fed into other data systems for serving we new the system would have to be able to guarantee fault-tolerance in the presence of machine failures. +

                  +Supporting these uses led use to a design with a number of unique elements, more akin to a database log then a traditional messaging system. We will outline some elements of the design in the following sections. + +

                  4.2 Persistence

                  +

                  Don't fear the filesystem!

                  +

                  +Kafka relies heavily on the filesystem for storing and caching messages. There is a general perception that "disks are slow" which makes people skeptical that a persistent structure can offer competitive performance. In fact disks are both much slower and much faster than people expect depending on how they are used; and a properly designed disk structure can often be as fast as the network. +

                  +The key fact about disk performance is that the throughput of hard drives has been diverging from the latency of a disk seek for the last decade. As a result the performance of linear writes on a JBOD configuration with six 7200rpm SATA RAID-5 array is about 600MB/sec but the performance of random writes is only about 100k/sec—a difference of over 6000X. These linear reads and writes are the most predictable of all usage patterns, and are heavily optimized by the operating system. A modern operating system provides read-ahead and write-behind techniques that prefetch data in large block multiples and group smaller logical writes into large physical writes. A further discussion of this issue can be found in this ACM Queue article; they actually find that sequential disk access can in some cases be faster than random memory access! +

                  +To compensate for this performance divergence modern operating systems have become increasingly aggressive in their use of main memory for disk caching. A modern OS will happily divert all free memory to disk caching with little performance penalty when the memory is reclaimed. All disk reads and writes will go through this unified cache. This feature cannot easily be turned off without using direct I/O, so even if a process maintains an in-process cache of the data, this data will likely be duplicated in OS pagecache, effectively storing everything twice. +

                  +Furthermore we are building on top of the JVM, and anyone who has spent any time with Java memory usage knows two things: +

                    +
                  1. The memory overhead of objects is very high, often doubling the size of the data stored (or worse).
                  2. +
                  3. Java garbage collection becomes increasingly fiddly and slow as the in-heap data increases.
                  4. +
                  +

                  +As a result of these factors using the filesystem and relying on pagecache is superior to maintaining an in-memory cache or other structure—we at least double the available cache by having automatic access to all free memory, and likely double again by storing a compact byte structure rather than individual objects. Doing so will result in a cache of up to 28-30GB on a 32GB machine without GC penalties. Furthermore this cache will stay warm even if the service is restarted, whereas the in-process cache will need to be rebuilt in memory (which for a 10GB cache may take 10 minutes) or else it will need to start with a completely cold cache (which likely means terrible initial performance). This also greatly simplifies the code as all logic for maintaining coherency between the cache and filesystem is now in the OS, which tends to do so more efficiently and more correctly than one-off in-process attempts. If your disk usage favors linear reads then read-ahead is effectively pre-populating this cache with useful data on each disk read. +

                  +This suggests a design which is very simple: rather than maintain as much as possible in-memory and flush it all out to the filesystem in a panic when we run out of space, we invert that. All data is immediately written to a persistent log on the filesystem without necessarily flushing to disk. In effect this just means that it is transferred into the kernel's pagecache. +

                  +This style of pagecache-centric design is described in an article on the design of Varnish here (along with a healthy dose of arrogance). + +

                  Constant Time Suffices

                  +

                  +The persistent data structure used in messaging systems are often a per-consumer queue with an associated BTree or other general-purpose random access data structures to maintain metadata about messages. BTrees are the most versatile data structure available, and make it possible to support a wide variety of transactional and non-transactional semantics in the messaging system. They do come with a fairly high cost, though: Btree operations are O(log N). Normally O(log N) is considered essentially equivalent to constant time, but this is not true for disk operations. Disk seeks come at 10 ms a pop, and each disk can do only one seek at a time so parallelism is limited. Hence even a handful of disk seeks leads to very high overhead. Since storage systems mix very fast cached operations with very slow physical disk operations, the observed performance of tree structures is often superlinear as data increases with fixed cache--i.e. doubling your data makes things much worse then twice as slow. +

                  +Intuitively a persistent queue could be built on simple reads and appends to files as is commonly the case with logging solutions. This structure has the advantage that all operations are O(1) and reads do not block writes or each other. This has obvious performance advantages since the performance is completely decoupled from the data size—one server can now take full advantage of a number of cheap, low-rotational speed 1+TB SATA drives. Though they have poor seek performance, these drives have acceptable performance for large reads and writes and come at 1/3 the price and 3x the capacity. +

                  +Having access to virtually unlimited disk space without any performance penalty means that we can provide some features not usually found in a messaging system. For example, in Kafka, instead of attempting to deleting messages as soon as they are consumed, we can retain messages for a relative long period (say a week). This leads to a great deal of flexibility for consumers, as we will describe. + +

                  4.3 Efficiency

                  +

                  +We have put significant effort into efficiency. One of our primary use cases is handling web activity data, which is very high volume: each page view may generate dozens of writes. Furthermore we assume each message published is read by at least one consumer (often many), hence we strive to make consumption as cheap as possible. +

                  +We have also found, from experience building and running a number of similar systems, that efficiency is a key to effective multi-tenant operations. If the downstream infrastructure service can easily become a bottleneck due to a small bump in usage by the application, such small changes will often create problems. By being very fast we help ensure that the application will tip-over under load before the infrastructure. This is particularly important when trying to run a centralized service that supports dozens or hundreds of applications on a centralized cluster as changes in usage patterns are a near-daily occurrence. +

                  +We discussed disk efficiency in the previous section. Once poor disk access patterns have been eliminated, there are two common causes of inefficiency in this type of system: too many small I/O operations, and excessive byte copying. +

                  +The small I/O problem happens both between the client and the server and in the server's own persistent operations. +

                  +To avoid this, our protocol is built around a "message set" abstraction that naturally groups messages together. This allows network requests to group messages together and amortize the overhead of the network roundtrip rather than sending a single message at a time. The server in turn appends chunks of messages to its log in one go, and the consumer fetches large linear chunks at a time. +

                  +This simple optimization produces orders of magnitude speed up. Batching leads to larger network packets, larger sequential disk operations, contiguous memory blocks, and so on, all of which allows Kafka to turn a bursty stream of random message writes into linear writes that flow to the consumers. +

                  +The other inefficiency is in byte copying. At low message rates this is not an issue, but under load the impact is significant. To avoid this we employ a standardized binary message format that is shared by the producer, the broker, and the consumer (so data chunks can be transferred without modification between them). +

                  +The message log maintained by the broker is itself just a directory of files, each populated by a sequence of message sets that have been written to disk in the same format used by the producer and consumer. Maintaining this common format allows optimization of the most important operation: network transfer of persistent log chunks. Modern unix operating systems offer a highly optimized code path for transferring data out of pagecache to a socket; in Linux this is done with the sendfile system call. +

                  +To understand the impact of sendfile, it is important to understand the common data path for transfer of data from file to socket: +

                    +
                  1. The operating system reads data from the disk into pagecache in kernel space
                  2. +
                  3. The application reads the data from kernel space into a user-space buffer
                  4. +
                  5. The application writes the data back into kernel space into a socket buffer
                  6. +
                  7. The operating system copies the data from the socket buffer to the NIC buffer where it is sent over the network
                  8. +
                  +

                  +This is clearly inefficient, there are four copies and two system calls. Using sendfile, this re-copying is avoided by allowing the OS to send the data from pagecache to the network directly. So in this optimized path, only the final copy to the NIC buffer is needed. +

                  +We expect a common use case to be multiple consumers on a topic. Using the zero-copy optimization above, data is copied into pagecache exactly once and reused on each consumption instead of being stored in memory and copied out to kernel space every time it is read. This allows messages to be consumed at a rate that approaches the limit of the network connection. +

                  +This combination of pagecache and sendfile means that on a Kafka cluster where the consumers are mostly caught up you will see no read activity on the disks whatsoever as they will be serving data entirely from cache. +

                  +For more background on the sendfile and zero-copy support in Java, see this article. + +

                  End-to-end Batch Compression

                  +

                  +In some cases the bottleneck is actually not CPU or disk but network bandwidth. This is particularly true for a data pipeline that needs to send messages between data centers over a wide-area network. Of course the user can always compress its messages one at a time without any support needed from Kafka, but this can lead to very poor compression ratios as much of the redundancy is due to repetition between messages of the same type (e.g. field names in JSON or user agents in web logs or common string values). Efficient compression requires compressing multiple messages together rather than compressing each message individually. +

                  +Kafka supports this by allowing recursive message sets. A batch of messages can be clumped together compressed and sent to the server in this form. This batch of messages will be written in compressed form and will remain compressed in the log and will only be decompressed by the consumer. +

                  +Kafka supports GZIP and Snappy compression protocols. More details on compression can be found here. + +

                  4.4 The Producer

                  + +

                  Load balancing

                  +

                  +The producer sends data directly to the broker that is the leader for the partition without any intervening routing tier. To help the producer do this all Kafka nodes can answer a request for metadata about which servers are alive and where the leaders for the partitions of a topic are at any given time to allow the producer to appropriate direct its requests. +

                  +The client controls which partition it publishes messages to. This can be done at random, implementing a kind of random load balancing, or it can be done by some semantic partitioning function. We expose the interface for semantic partitioning by allowing the user to specify a key to partition by and using this to hash to a partition (there is also an option to override the partition function if need be). For example if the key chosen was a user id then all data for a given user would be sent to the same partition. This in turn will allow consumers to make locality assumptions about their consumption. This style of partitioning is explicitly designed to allow locality-sensitive processing in consumers. + +

                  Asynchronous send

                  +

                  +Batching is one of the big drivers of efficiency, and to enable batching the Kafka producer will attempt to accumulate data in memory and to send out larger batches in a single request. The batching can be configured to accumulate no more than a fixed number of messages and to wait no longer than some fixed latency bound (say 64k or 10 ms). This allows the accumulation of more bytes to send, and few larger I/O operations on the servers. This buffering is configurable and gives a mechanism to trade off a small amount of additional latency for better throughput. +

                  +Details on configuration and api for the producer can be found elsewhere in the documentation. + +

                  4.5 The Consumer

                  + +The Kafka consumer works by issuing "fetch" requests to the brokers leading the partitions it wants to consume. The consumer specifies its offset in the log with each request and receives back a chunk of log beginning from that position. The consumer thus has significant control over this position and can rewind it to re-consume data if need be. + +

                  Push vs. pull

                  +

                  +An initial question we considered is whether consumers should pull data from brokers or brokers should push data to the consumer. In this respect Kafka follows a more traditional design, shared by most messaging systems, where data is pushed to the broker from the producer and pulled from the broker by the consumer. Some logging-centric systems, such as Scribe and Apache Flume follow a very different push based path where data is pushed downstream. There are pros and cons to both approaches. However a push-based system has difficulty dealing with diverse consumers as the broker controls the rate at which data is transferred. The goal is generally for the consumer to be able to consume at the maximum possible rate; unfortunately in a push system this means the consumer tends to be overwhelmed when its rate of consumption falls below the rate of production (a denial of service attack, in essence). A pull-based system has the nicer property that the consumer simply falls behind and catches up when it can. This can be mitigated with some kind of backoff protocol by which the consumer can indicate it is overwhelmed, but getting the rate of transfer to fully utilize (but never over-utilize) the consumer is trickier than it seems. Previous attempts at building systems in this fashion led us to go with a more traditional pull model. +

                  +Another advantage of a pull-based system is that it lends itself to aggressive batching of data sent to the consumer. A push-based system must choose to either send a request immediately or accumulate more data and then send it later without knowledge of whether the downstream consumer will be able to immediately process it. If tuned for low latency this will result in sending a single message at a time only for the transfer to end up being buffered anyway, which is wasteful. A pull-based design fixes this as the consumer always pulls all available messages after its current position in the log (or up to some configurable max size). So one gets optimal batching without introducing unnecessary latency. +

                  +The deficiency of a naive pull-based system is that if the broker has no data the consumer may end up polling in a tight loop, effectively busy-waiting for data to arrive. To avoid this we have parameters in our pull request that allow the consumer request to block in a "long poll" waiting until data arrives (and optionally waiting until a given number of bytes is available to ensure large transfer sizes). +

                  +You could imagine other possible designs which would be only pull, end-to-end. The producer would locally write to a local log, and brokers would pull from that with consumers pulling from them. A similar type of "store-and-forward" producer is often proposed. This is intriguing but we felt not very suitable for our target use cases which have thousands of producers. Our experience running persistent data systems at scale led us to feel that involving thousands of disks in the system across many applications would not actually make things more reliable and would be a nightmare to operate. And in practice we have found that we can run a pipeline with strong SLAs at large scale without a need for producer persistence. + +

                  Consumer Position

                  +Keeping track of what has been consumed, is, surprisingly, one of the key performance points of a messaging system. +

                  +Most messaging systems keep metadata about what messages have been consumed on the broker. That is, as a message is handed out to a consumer, the broker either records that fact locally immediately or it may wait for acknowledgement from the consumer. This is a fairly intuitive choice, and indeed for a single machine server it is not clear where else this state could go. Since the data structure used for storage in many messaging systems scale poorly, this is also a pragmatic choice--since the broker knows what is consumed it can immediately delete it, keeping the data size small. +

                  +What is perhaps not obvious, is that getting the broker and consumer to come into agreement about what has been consumed is not a trivial problem. If the broker records a message as consumed immediately every time it is handed out over the network, then if the consumer fails to process the message (say because it crashes or the request times out or whatever) that message will be lost. To solve this problem, many messaging systems add an acknowledgement feature which means that messages are only marked as sent not consumed when they are sent; the broker waits for a specific acknowledgement from the consumer to record the message as consumed. This strategy fixes the problem of losing messages, but creates new problems. First of all, if the consumer processes the message but fails before it can send an acknowledgement then the message will be consumed twice. The second problem is around performance, now the broker must keep multiple states about every single message (first to lock it so it is not given out a second time, and then to mark it as permanently consumed so that it can be removed). Tricky problems must be dealt with, like what to do with messages that are sent but never acknowledged. +

                  +Kafka handles this differently. Our topic is divided into a set of totally ordered partitions, each of which is consumed by one consumer at any given time. This means that the position of consumer in each partition is just a single integer, the offset of the next message to consume. This makes the state about what has been consumed very small, just one number for each partition. This state can be periodically checkpointed. This makes the equivalent of message acknowledgements very cheap. +

                  +There is a side benefit of this decision. A consumer can deliberately rewind back to an old offset and re-consume data. This violates the common contract of a queue, but turns out to be an essential feature for many consumers. For example, if the consumer code has a bug and is discovered after some messages are consumed, the consumer can re-consume those messages once the bug is fixed. + +

                  Offline Data Load

                  + +Scalable persistence allows for the possibility of consumers that only periodically consume such as batch data loads that periodically bulk-load data into an offline system such as Hadoop or a relational data warehouse. +

                  +In the case of Hadoop we parallelize the data load by splitting the load over individual map tasks, one for each node/topic/partition combination, allowing full parallelism in the loading. Hadoop provides the task management, and tasks which fail can restart without danger of duplicate data—they simply restart from their original position. + +

                  4.6 Message Delivery Semantics

                  +

                  +Now that we understand a little about how producers and consumers work, let's discuss the semantic guarantees Kafka provides between producer and consumer. Clearly there are multiple possible message delivery guarantees that could be provided: +

                    +
                  • + At most once—Messages may be lost but are never redelivered. +
                  • +
                  • + At least once—Messages are never lost but may be redelivered. +
                  • +
                  • + Exactly once—this is what people actually want, each message is delivered once and only once. +
                  • +
                  + +It's worth noting that this breaks down into two problems: the durability guarantees for publishing a message and the guarantees when consuming a message. +

                  +Many systems claim to provide "exactly once" delivery semantics, but it is important to read the fine print, most of these claims are misleading (i.e. they don't translate to the case where consumers or producers can fail, or cases where there are multiple consumer processes, or cases where data written to disk can be lost). +

                  +Kafka's semantics are straight-forward. When publishing a message we have a notion of the message being "committed" to the log. Once a published message is committed it will not be lost as long as one broker that replicates the partition to which this message was written remains "alive". The definition of alive as well as a description of which types of failures we attempt to handle will be described in more detail in the next section. For now let's assume a perfect, lossless broker and try to understand the guarantees to the producer and consumer. If a producer attempts to publish a message and experiences a network error it cannot be sure if this error happened before or after the message was committed. This is similar to the semantics of inserting into a database table with an autogenerated key. +

                  +These are not the strongest possible semantics for publishers. Although we cannot be sure of what happened in the case of a network error, it is possible to allow the producer to generate a sort of "primary key" that makes retrying the produce request idempotent. This feature is not trivial for a replicated system because of course it must work even (or especially) in the case of a server failure. With this feature it would suffice for the producer to retry until it receives acknowledgement of a successfully committed message at which point we would guarantee the message had been published exactly once. We hope to add this in a future Kafka version. +

                  +Not all use cases require such strong guarantees. For uses which are latency sensitive we allow the producer to specify the durability level it desires. If the producer specifies that it wants to wait on the message being committed this can take on the order of 10 ms. However the producer can also specify that it wants to perform the send completely asynchronously or that it wants to wait only until the leader (but not necessarily the followers) have the message. +

                  +Now let's describe the semantics from the point-of-view of the consumer. All replicas have the exact same log with the same offsets. The consumer controls its position in this log. If the consumer never crashed it could just store this position in memory, but if the consumer fails and we want this topic partition to be taken over by another process the new process will need to choose an appropriate position from which to start processing. Let's say the consumer reads some messages -- it has several options for processing the messages and updating its position. +

                    +
                  1. It can read the messages, then save its position in the log, and finally process the messages. In this case there is a possibility that the consumer process crashes after saving its position but before saving the output of its message processing. In this case the process that took over processing would start at the saved position even though a few messages prior to that position had not been processed. This corresponds to "at-most-once" semantics as in the case of a consumer failure messages may not be processed. +
                  2. It can read the messages, process the messages, and finally save its position. In this case there is a possibility that the consumer process crashes after processing messages but before saving its position. In this case when the new process takes over the first few messages it receives will already have been processed. This corresponds to the "at-least-once" semantics in the case of consumer failure. In many cases messages have a primary key and so the updates are idempotent (receiving the same message twice just overwrites a record with another copy of itself). +
                  3. So what about exactly once semantics (i.e. the thing you actually want)? The limitation here is not actually a feature of the messaging system but rather the need to co-ordinate the consumer's position with what is actually stored as output. The classic way of achieving this would be to introduce a two-phase commit between the storage for the consumer position and the storage of the consumers output. But this can be handled more simply and generally by simply letting the consumer store its offset in the same place as its output. This is better because many of the output systems a consumer might want to write to will not support a two-phase commit. As an example of this, our Hadoop ETL that populates data in HDFS stores its offsets in HDFS with the data it reads so that it is guaranteed that either data and offsets are both updated or neither is. We follow similar patterns for many other data systems which require these stronger semantics and for which the messages do not have a primary key to allow for deduplication. +
                  +

                  +So effectively Kafka guarantees at-least-once delivery by default and allows the user to implement at most once delivery by disabling retries on the producer and committing its offset prior to processing a batch of messages. Exactly-once delivery requires co-operation with the destination storage system but Kafka provides the offset which makes implementing this straight-forward. + +

                  4.7 Replication

                  +

                  +Kafka replicates the log for each topic's partitions across a configurable number of servers (you can set this replication factor on a topic-by-topic basis). This allows automatic failover to these replicas when a server in the cluster fails so messages remain available in the presence of failures. +

                  +Other messaging systems provide some replication-related features, but, in our (totally biased) opinion, this appears to be a tacked-on thing, not heavily used, and with large downsides: slaves are inactive, throughput is heavily impacted, it requires fiddly manual configuration, etc. Kafka is meant to be used with replication by default—in fact we implement un-replicated topics as replicated topics where the replication factor is one. +

                  +The unit of replication is the topic partition. Under non-failure conditions, each partition in Kafka has a single leader and zero or more followers. The total number of replicas including the leader constitute the replication factor. All reads and writes go to the leader of the partition. Typically, there are many more partitions than brokers and the leaders are evenly distributed among brokers. The logs on the followers are identical to the leader's log—all have the same offsets and messages in the same order (though, of course, at any given time the leader may have a few as-yet unreplicated messages at the end of its log). +

                  +Followers consume messages from the leader just as a normal Kafka consumer would and apply them to their own log. Having the followers pull from the leader has the nice property of allowing the follower to naturally batch together log entries they are applying to their log. +

                  +As with most distributed systems automatically handling failures requires having a precise definition of what it means for a node to be "alive". For Kafka node liveness has two conditions +

                    +
                  1. A node must be able to maintain its session with ZooKeeper (via ZooKeeper's heartbeat mechanism) +
                  2. If it is a slave it must replicate the writes happening on the leader and not fall "too far" behind +
                  +We refer to nodes satisfying these two conditions as being "in sync" to avoid the vagueness of "alive" or "failed". The leader keeps track of the set of "in sync" nodes. If a follower dies, gets stuck, or falls behind, the leader will remove it from the list of in sync replicas. The determination of stuck and lagging replicas is controlled by the replica.lag.time.max.ms configuration. +

                  +In distributed systems terminology we only attempt to handle a "fail/recover" model of failures where nodes suddenly cease working and then later recover (perhaps without knowing that they have died). Kafka does not handle so-called "Byzantine" failures in which nodes produce arbitrary or malicious responses (perhaps due to bugs or foul play). +

                  +A message is considered "committed" when all in sync replicas for that partition have applied it to their log. Only committed messages are ever given out to the consumer. This means that the consumer need not worry about potentially seeing a message that could be lost if the leader fails. Producers, on the other hand, have the option of either waiting for the message to be committed or not, depending on their preference for tradeoff between latency and durability. This preference is controlled by the request.required.acks setting that the producer uses. +

                  +The guarantee that Kafka offers is that a committed message will not be lost, as long as there is at least one in sync replica alive, at all times. +

                  +Kafka will remain available in the presence of node failures after a short fail-over period, but may not remain available in the presence of network partitions. + +

                  Replicated Logs: Quorums, ISRs, and State Machines (Oh my!)

                  + +At its heart a Kafka partition is a replicated log. The replicated log is one of the most basic primitives in distributed data systems, and there are many approaches for implementing one. A replicated log can be used by other systems as a primitive for implementing other distributed systems in the state-machine style. +

                  +A replicated log models the process of coming into consensus on the order of a series of values (generally numbering the log entries 0, 1, 2, ...). There are many ways to implement this, but the simplest and fastest is with a leader who chooses the ordering of values provided to it. As long as the leader remains alive, all followers need to only copy the values and ordering, the leader chooses. +

                  +Of course if leaders didn't fail we wouldn't need followers! When the leader does die we need to choose a new leader from among the followers. But followers themselves may fall behind or crash so we must ensure we choose an up-to-date follower. The fundamental guarantee a log replication algorithm must provide is that if we tell the client a message is committed, and the leader fails, the new leader we elect must also have that message. This yields a tradeoff: if the leader waits for more followers to acknowledge a message before declaring it committed then there will be more potentially electable leaders. +

                  +If you choose the number of acknowledgements required and the number of logs that must be compared to elect a leader such that there is guaranteed to be an overlap, then this is called a Quorum. +

                  +A common approach to this tradeoff is to use a majority vote for both the commit decision and the leader election. This is not what Kafka does, but let's explore it anyway to understand the tradeoffs. Let's say we have 2f+1 replicas. If f+1 replicas must receive a message prior to a commit being declared by the leader, and if we elect a new leader by electing the follower with the most complete log from at least f+1 replicas, then, with no more than f failures, the leader is guaranteed to have all committed messages. This is because among any f+1 replicas, there must be at least one replica that contains all committed messages. That replica's log will be the most complete and therefore will be selected as the new leader. There are many remaining details that each algorithm must handle (such as precisely defined what makes a log more complete, ensuring log consistency during leader failure or changing the set of servers in the replica set) but we will ignore these for now. +

                  +This majority vote approach has a very nice property: the latency is dependent on only the fastest servers. That is, if the replication factor is three, the latency is determined by the faster slave not the slower one. +

                  +There are a rich variety of algorithms in this family including ZooKeeper's Zab, Raft, and Viewstamped Replication. The most similar academic publication we are aware of to Kafka's actual implementation is PacificA from Microsoft. +

                  +The downside of majority vote is that it doesn't take many failures to leave you with no electable leaders. To tolerate one failure requires three copies of the data, and to tolerate two failures requires five copies of the data. In our experience having only enough redundancy to tolerate a single failure is not enough for a practical system, but doing every write five times, with 5x the disk space requirements and 1/5th the throughput, is not very practical for large volume data problems. This is likely why quorum algorithms more commonly appear for shared cluster configuration such as ZooKeeper but are less common for primary data storage. For example in HDFS the namenode's high-availability feature is built on a majority-vote-based journal, but this more expensive approach is not used for the data itself. +

                  +Kafka takes a slightly different approach to choosing its quorum set. Instead of majority vote, Kafka dynamically maintains a set of in-sync replicas (ISR) that are caught-up to the leader. Only members of this set are eligible for election as leader. A write to a Kafka partition is not considered committed until all in-sync replicas have received the write. This ISR set is persisted to ZooKeeper whenever it changes. Because of this, any replica in the ISR is eligible to be elected leader. This is an important factor for Kafka's usage model where there are many partitions and ensuring leadership balance is important. With this ISR model and f+1 replicas, a Kafka topic can tolerate f failures without losing committed messages. +

                  +For most use cases we hope to handle, we think this tradeoff is a reasonable one. In practice, to tolerate f failures, both the majority vote and the ISR approach will wait for the same number of replicas to acknowledge before committing a message (e.g. to survive one failure a majority quorum needs three replicas and one acknowledgement and the ISR approach requires two replicas and one acknowledgement). The ability to commit without the slowest servers is an advantage of the majority vote approach. However, we think it is ameliorated by allowing the client to choose whether they block on the message commit or not, and the additional throughput and disk space due to the lower required replication factor is worth it. +

                  +Another important design distinction is that Kafka does not require that crashed nodes recover with all their data intact. It is not uncommon for replication algorithms in this space to depend on the existence of "stable storage" that cannot be lost in any failure-recovery scenario without potential consistency violations. There are two primary problems with this assumption. First, disk errors are the most common problem we observe in real operation of persistent data systems and they often do not leave data intact. Secondly, even if this were not a problem, we do not want to require the use of fsync on every write for our consistency guarantees as this can reduce performance by two to three orders of magnitude. Our protocol for allowing a replica to rejoin the ISR ensures that before rejoining, it must fully re-sync again even if it lost unflushed data in its crash. + +

                  Unclean leader election: What if they all die?

                  + +Note that Kafka's guarantee with respect to data loss is predicated on at least on replica remaining in sync. If all the nodes replicating a partition die, this guarantee no longer holds. +

                  +However a practical system needs to do something reasonable when all the replicas die. If you are unlucky enough to have this occur, it is important to consider what will happen. There are two behaviors that could be implemented: +

                    +
                  1. Wait for a replica in the ISR to come back to life and choose this replica as the leader (hopefully it still has all its data). +
                  2. Choose the first replica (not necessarily in the ISR) that comes back to life as the leader. +
                  +

                  +This is a simple tradeoff between availability and consistency. If we wait for replicas in the ISR, then we will remain unavailable as long as those replicas are down. If such replicas were destroyed or their data was lost, then we are permanently down. If, on the other hand, a non-in-sync replica comes back to life and we allow it to become leader, then its log becomes the source of truth even though it is not guaranteed to have every committed message. In our current release we choose the second strategy and favor choosing a potentially inconsistent replica when all replicas in the ISR are dead. In the future, we would like to make this configurable to better support use cases where downtime is preferable to inconsistency. +

                  +This dilemma is not specific to Kafka. It exists in any quorum-based scheme. For example in a majority voting scheme, if a majority of servers suffer a permanent failure, then you must either choose to lose 100% of your data or violate consistency by taking what remains on an existing server as your new source of truth. + + +

                  Availability and Durability Guarantees

                  + +When writing to Kafka, producers can choose whether they wait for the message to be acknowledged by 0,1 or all (-1) replicas. +Note that "acknowledgement by all replicas" does not guarantee that the full set of assigned replicas have received the message. By default, when request.required.acks=-1, acknowledgement happens as soon as all the current in-sync replicas have received the message. For example, if a topic is configured with only two replicas and one fails (i.e., only one in sync replica remains), then writes that specify request.required.acks=-1 will succeed. However, these writes could be lost if the remaining replica also fails. + +Although this ensures maximum availability of the partition, this behavior may be undesirable to some users who prefer durability over availability. Therefore, we provide two topic-level configurations that can be used to prefer message durability over availability: +
                    +
                  1. Disable unclean leader election - if all replicas become unavailable, then the partition will remain unavailable until the most recent leader becomes available again. This effectively prefers unavailability over the risk of message loss. See the previous section on Unclean Leader Election for clarification.
                  2. +
                  3. Specify a minimum ISR size - the partition will only accept writes if the size of the ISR is above a certain minimum, in order to prevent the loss of messages that were written to just a single replica, which subsequently becomes unavailable. This setting only takes effect if the producer uses required.acks=-1 and guarantees that the message will be acknowledged by at least this many in-sync replicas. +This setting offers a trade-off between consistency and availability. A higher setting for minimum ISR size guarantees better consistency since the message is guaranteed to be written to more replicas which reduces the probability that it will be lost. However, it reduces availability since the partition will be unavailable for writes if the number of in-sync replicas drops below the minimum threshold.
                  4. +
                  + + +

                  Replica Management

                  + +The above discussion on replicated logs really covers only a single log, i.e. one topic partition. However a Kafka cluster will manage hundreds or thousands of these partitions. We attempt to balance partitions within a cluster in a round-robin fashion to avoid clustering all partitions for high-volume topics on a small number of nodes. Likewise we try to balance leadership so that each node is the leader for a proportional share of its partitions. +

                  +It is also important to optimize the leadership election process as that is the critical window of unavailability. A naive implementation of leader election would end up running an election per partition for all partitions a node hosted when that node failed. Instead, we elect one of the brokers as the "controller". This controller detects failures at the broker level and is responsible for changing the leader of all affected partitions in a failed broker. The result is that we are able to batch together many of the required leadership change notifications which makes the election process far cheaper and faster for a large number of partitions. If the controller fails, one of the surviving brokers will become the new controller. + +

                  4.8 Log Compaction

                  + +Log compaction ensures that Kafka will always retain at least the last known value for each message key within the log of data for a single topic partition. It addresses use cases and scenarios such as restoring state after application crashes or system failure, or reloading caches after application restarts during operational maintenance. Let's dive into these use cases in more detail and then describe how compaction works. +

                  +So far we have described only the simpler approach to data retention where old log data is discarded after a fixed period of time or when the log reaches some predetermined size. This works well for temporal event data such as logging where each record stands alone. However an important class of data streams are the log of changes to keyed, mutable data (for example, the changes to a database table). +

                  +Let's discuss a concrete example of such a stream. Say we have a topic containing user email addresses; every time a user updates their email address we send a message to this topic using their user id as the primary key. Now say we send the following messages over some time period for a user with id 123, each message corresponding to a change in email address (messages for other ids are omitted): +

                  +    123 => bill@microsoft.com
                  +            .
                  +            .
                  +            .
                  +    123 => bill@gatesfoundation.org
                  +            .
                  +            .
                  +            .
                  +    123 => bill@gmail.com
                  +
                  +Log compaction gives us a more granular retention mechanism so that we are guaranteed to retain at least the last update for each primary key (e.g. bill@gmail.com). By doing this we guarantee that the log contains a full snapshot of the final value for every key not just keys that changed recently. This means downstream consumers can restore their own state off this topic without us having to retain a complete log of all changes. +

                  +Let's start by looking at a few use cases where this is useful, then we'll see how it can be used. +

                    +
                  1. Database change subscription. It is often necessary to have a data set in multiple data systems, and often one of these systems is a database of some kind (either a RDBMS or perhaps a new-fangled key-value store). For example you might have a database, a cache, a search cluster, and a Hadoop cluster. Each change to the database will need to be reflected in the cache, the search cluster, and eventually in Hadoop. In the case that one is only handling the real-time updates you only need recent log. But if you want to be able to reload the cache or restore a failed search node you may need a complete data set. +
                  2. Event sourcing. This is a style of application design which co-locates query processing with application design and uses a log of changes as the primary store for the application. +
                  3. Journaling for high-availability. A process that does local computation can be made fault-tolerant by logging out changes that it makes to it's local state so another process can reload these changes and carry on if it should fail. A concrete example of this is handling counts, aggregations, and other "group by"-like processing in a stream query system. Samza, a real-time stream-processing framework, uses this feature for exactly this purpose. +
                  +In each of these cases one needs primarily to handle the real-time feed of changes, but occasionally, when a machine crashes or data needs to be re-loaded or re-processed, one needs to do a full load. Log compaction allows feeding both of these use cases off the same backing topic. + +This style of usage of a log is described in more detail in this blog post. +

                  +The general idea is quite simple. If we had infinite log retention, and we logged each change in the above cases, then we would have captured the state of the system at each time from when it first began. Using this complete log we could restore to any point in time by replaying the first N records in the log. This hypothetical complete log is not very practical for systems that update a single record many times as the log will grow without bound even for a stable dataset. The simple log retention mechanism which throws away old updates will bound space but the log is no longer a way to restore the current state—now restoring from the beginning of the log no longer recreates the current state as old updates may not be captured at all. +

                  +Log compaction is a mechanism to give finer-grained per-record retention, rather than the coarser-grained time-based retention. The idea is to selectively remove records where we have a more recent update with the same primary key. This way the log is guaranteed to have at least the last state for each key. +

                  +This retention policy can be set per-topic, so a single cluster can have some topics where retention is enforced by size or time and other topics where retention is enforced by compaction. +

                  +This functionality is inspired by one of LinkedIn's oldest and most successful pieces of infrastructure—a database changelog caching service called Databus. Unlike most log-structured storage systems Kafka is built for subscription and organizes data for fast linear reads and writes. Unlike Databus, Kafka acts a source-of-truth store so it is useful even in situations where the upstream data source would not otherwise be replayable. + +

                  Log Compaction Basics

                  + +Here is a high-level picture that shows the logical structure of a Kafka log with the offset for each message. +

                  + +

                  +The head of the log is identical to a traditional Kafka log. It has dense, sequential offsets and retains all messages. Log compaction adds an option for handling the tail of the log. The picture above shows a log with a compacted tail. Note that the messages in the tail of the log retain the original offset assigned when they were first written—that never changes. Note also that all offsets remain valid positions in the log, even if the message with that offset has been compacted away; in this case this position is indistinguishable from the next highest offset that does appear in the log. For example, in the picture above the offsets 36, 37, and 38 are all equivalent positions and a read beginning at any of these offsets would return a message set beginning with 38. +

                  +Compaction also allows for deletes. A message with a key and a null payload will be treated as a delete from the log. This delete marker will cause any prior message with that key to be removed (as would any new message with that key), but delete markers are special in that they will themselves be cleaned out of the log after a period of time to free up space. The point in time at which deletes are no longer retained is marked as the "delete retention point" in the above diagram. +

                  +The compaction is done in the background by periodically recopying log segments. Cleaning does not block reads and can be throttled to use no more than a configurable amount of I/O throughput to avoid impacting producers and consumers. The actual process of compacting a log segment looks something like this: +

                  + +

                  +

                  What guarantees does log compaction provide?

                  + +Log compaction guarantees the following: +
                    +
                  1. Any consumer that stays caught-up to within the head of the log will see every message that is written; these messages will have sequential offsets. +
                  2. Ordering of messages is always maintained. Compaction will never re-order messages, just remove some. +
                  3. The offset for a message never changes. It is the permanent identifier for a position in the log. +
                  4. Any read progressing from offset 0 will see at least the final state of all records in the order they were written. All delete markers for deleted records will be seen provided the reader reaches the head of the log in a time period less than the topic's delete.retention.ms setting (the default is 24 hours). This is important as delete marker removal happens concurrently with read (and thus it is important that we not remove any delete marker prior to the reader seeing it). +
                  5. Any consumer progressing from the start of the log, will see at least the final state of all records in the order they were written. All delete markers for deleted records will be seen provided the consumer reaches the head of the log in a time period less than the topic's delete.retention.ms setting (the default is 24 hours). This is important as delete marker removal happens concurrently with read, and thus it is important that we do not remove any delete marker prior to the consumer seeing it. +
                  + +

                  Log Compaction Details

                  + +Log compaction is handled by the log cleaner, a pool of background threads that recopy log segment files, removing records whose key appears in the head of the log. Each compactor thread works as follows: +
                    +
                  1. It chooses the log that has the highest ratio of log head to log tail +
                  2. It creates a succinct summary of the last offset for each key in the head of the log +
                  3. It recopies the log from beginning to end removing keys which have a later occurrence in the log. New, clean segments are swapped into the log immediately so the additional disk space required is just one additional log segment (not a fully copy of the log). +
                  4. The summary of the log head is essentially just a space-compact hash table. It uses exactly 24 bytes per entry. As a result with 8GB of cleaner buffer one cleaner iteration can clean around 366GB of log head (assuming 1k messages). +
                  +

                  +

                  Configuring The Log Cleaner

                  + +The log cleaner is disabled by default. To enable it set the server config +
                    log.cleaner.enable=true
                  +This will start the pool of cleaner threads. To enable log cleaning on a particular topic you can add the log-specific property +
                    log.cleanup.policy=compact
                  +This can be done either at topic creation time or using the alter topic command. +

                  +Further cleaner configurations are described here. + +

                  Log Compaction Limitations

                  + +
                    +
                  1. You cannot configure yet how much log is retained without compaction (the "head" of the log). Currently all segments are eligible except for the last segment, i.e. the one currently being written to.
                  2. +
                  3. Log compaction is not yet compatible with compressed topics.
                  4. +
                  diff --git a/docs/documentation.html b/docs/documentation.html new file mode 100644 index 0000000000000..8f9b081d03ec3 --- /dev/null +++ b/docs/documentation.html @@ -0,0 +1,133 @@ + + + + +

                  Kafka 0.8.2 Documentation

                  +Prior releases: 0.7.x, 0.8.0, 0.8.1.X. + + + + +

                  1. Getting Started

                  + + + + + + +

                  2. API

                  + + + +

                  3. Configuration

                  + + + +

                  4. Design

                  + + + +

                  5. Implementation

                  + + + +

                  6. Operations

                  + + + + diff --git a/docs/ecosystem.html b/docs/ecosystem.html new file mode 100644 index 0000000000000..e99a44608ac21 --- /dev/null +++ b/docs/ecosystem.html @@ -0,0 +1,20 @@ + + +

                  1.4 Ecosystem

                  + +There are a plethora of tools that integrate with Kafka outside the main distribution. The ecosystem page lists many of these, including stream processing systems, Hadoop integration, monitoring, and deployment tools. diff --git a/docs/implementation.html b/docs/implementation.html new file mode 100644 index 0000000000000..25f9b39843550 --- /dev/null +++ b/docs/implementation.html @@ -0,0 +1,387 @@ + + +

                  5.1 API Design

                  + +

                  Producer APIs

                  + +

                  +The Producer API that wraps the 2 low-level producers - kafka.producer.SyncProducer and kafka.producer.async.AsyncProducer. +

                  +class Producer {
                  +	
                  +  /* Sends the data, partitioned by key to the topic using either the */
                  +  /* synchronous or the asynchronous producer */
                  +  public void send(kafka.javaapi.producer.ProducerData<K,V> producerData);
                  +
                  +  /* Sends a list of data, partitioned by key to the topic using either */
                  +  /* the synchronous or the asynchronous producer */
                  +  public void send(java.util.List<kafka.javaapi.producer.ProducerData<K,V>> producerData);
                  +
                  +  /* Closes the producer and cleans up */	
                  +  public void close();
                  +
                  +}
                  +
                  + +The goal is to expose all the producer functionality through a single API to the client. + +The new producer - +
                    +
                  • can handle queueing/buffering of multiple producer requests and asynchronous dispatch of the batched data - +

                    kafka.producer.Producer provides the ability to batch multiple produce requests (producer.type=async), before serializing and dispatching them to the appropriate kafka broker partition. The size of the batch can be controlled by a few config parameters. As events enter a queue, they are buffered in a queue, until either queue.time or batch.size is reached. A background thread (kafka.producer.async.ProducerSendThread) dequeues the batch of data and lets the kafka.producer.EventHandler serialize and send the data to the appropriate kafka broker partition. A custom event handler can be plugged in through the event.handler config parameter. At various stages of this producer queue pipeline, it is helpful to be able to inject callbacks, either for plugging in custom logging/tracing code or custom monitoring logic. This is possible by implementing the kafka.producer.async.CallbackHandler interface and setting callback.handler config parameter to that class. +

                    +
                  • +
                  • handles the serialization of data through a user-specified Encoder - +
                    +interface Encoder<T> {
                    +  public Message toMessage(T data);
                    +}
                    +
                    +

                    The default is the no-op kafka.serializer.DefaultEncoder

                    +
                  • +
                  • provides software load balancing through an optionally user-specified Partitioner - +

                    +The routing decision is influenced by the kafka.producer.Partitioner. +

                    +interface Partitioner<T> {
                    +   int partition(T key, int numPartitions);
                    +}
                    +
                    +The partition API uses the key and the number of available broker partitions to return a partition id. This id is used as an index into a sorted list of broker_ids and partitions to pick a broker partition for the producer request. The default partitioning strategy is hash(key)%numPartitions. If the key is null, then a random broker partition is picked. A custom partitioning strategy can also be plugged in using the partitioner.class config parameter. +

                    +
                  • +
                  +

                  + +

                  Consumer APIs

                  +

                  +We have 2 levels of consumer APIs. The low-level "simple" API maintains a connection to a single broker and has a close correspondence to the network requests sent to the server. This API is completely stateless, with the offset being passed in on every request, allowing the user to maintain this metadata however they choose. +

                  +

                  +The high-level API hides the details of brokers from the consumer and allows consuming off the cluster of machines without concern for the underlying topology. It also maintains the state of what has been consumed. The high-level API also provides the ability to subscribe to topics that match a filter expression (i.e., either a whitelist or a blacklist regular expression). +

                  + +
                  Low-level API
                  +
                  +class SimpleConsumer {
                  +	
                  +  /* Send fetch request to a broker and get back a set of messages. */ 
                  +  public ByteBufferMessageSet fetch(FetchRequest request);
                  +
                  +  /* Send a list of fetch requests to a broker and get back a response set. */ 
                  +  public MultiFetchResponse multifetch(List<FetchRequest> fetches);
                  +
                  +  /**
                  +   * Get a list of valid offsets (up to maxSize) before the given time.
                  +   * The result is a list of offsets, in descending order.
                  +   * @param time: time in millisecs,
                  +   *              if set to OffsetRequest$.MODULE$.LATIEST_TIME(), get from the latest offset available.
                  +   *              if set to OffsetRequest$.MODULE$.EARLIEST_TIME(), get from the earliest offset available.
                  +   */
                  +  public long[] getOffsetsBefore(String topic, int partition, long time, int maxNumOffsets);
                  +}
                  +
                  + +The low-level API is used to implement the high-level API as well as being used directly for some of our offline consumers (such as the hadoop consumer) which have particular requirements around maintaining state. + +
                  High-level API
                  +
                  +
                  +/* create a connection to the cluster */ 
                  +ConsumerConnector connector = Consumer.create(consumerConfig);
                  +
                  +interface ConsumerConnector {
                  +	
                  +  /**
                  +   * This method is used to get a list of KafkaStreams, which are iterators over
                  +   * MessageAndMetadata objects from which you can obtain messages and their
                  +   * associated metadata (currently only topic).
                  +   *  Input: a map of <topic, #streams>
                  +   *  Output: a map of <topic, list of message streams>
                  +   */
                  +  public Map<String,List<KafkaStream>> createMessageStreams(Map<String,Int> topicCountMap); 
                  +
                  +  /**
                  +   * You can also obtain a list of KafkaStreams, that iterate over messages
                  +   * from topics that match a TopicFilter. (A TopicFilter encapsulates a
                  +   * whitelist or a blacklist which is a standard Java regex.)
                  +   */
                  +  public List<KafkaStream> createMessageStreamsByFilter(
                  +      TopicFilter topicFilter, int numStreams);
                  +
                  +  /* Commit the offsets of all messages consumed so far. */
                  +  public commitOffsets()
                  +  
                  +  /* Shut down the connector */
                  +  public shutdown()
                  +}
                  +
                  +

                  +This API is centered around iterators, implemented by the KafkaStream class. Each KafkaStream represents the stream of messages from one or more partitions on one or more servers. Each stream is used for single threaded processing, so the client can provide the number of desired streams in the create call. Thus a stream may represent the merging of multiple server partitions (to correspond to the number of processing threads), but each partition only goes to one stream. +

                  +

                  +The createMessageStreams call registers the consumer for the topic, which results in rebalancing the consumer/broker assignment. The API encourages creating many topic streams in a single call in order to minimize this rebalancing. The createMessageStreamsByFilter call (additionally) registers watchers to discover new topics that match its filter. Note that each stream that createMessageStreamsByFilter returns may iterate over messages from multiple topics (i.e., if multiple topics are allowed by the filter). +

                  + +

                  5.2 Network Layer

                  +

                  +The network layer is a fairly straight-forward NIO server, and will not be described in great detail. The sendfile implementation is done by giving the MessageSet interface a writeTo method. This allows the file-backed message set to use the more efficient transferTo implementation instead of an in-process buffered write. The threading model is a single acceptor thread and N processor threads which handle a fixed number of connections each. This design has been pretty thoroughly tested elsewhere and found to be simple to implement and fast. The protocol is kept quite simple to allow for future implementation of clients in other languages. +

                  +

                  5.3 Messages

                  +

                  +Messages consist of a fixed-size header and variable length opaque byte array payload. The header contains a format version and a CRC32 checksum to detect corruption or truncation. Leaving the payload opaque is the right decision: there is a great deal of progress being made on serialization libraries right now, and any particular choice is unlikely to be right for all uses. Needless to say a particular application using Kafka would likely mandate a particular serialization type as part of its usage. The MessageSet interface is simply an iterator over messages with specialized methods for bulk reading and writing to an NIO Channel. + +

                  5.4 Message Format

                  + +
                  +	/** 
                  +	 * A message. The format of an N byte message is the following: 
                  +	 * 
                  +	 * If magic byte is 0 
                  +	 * 
                  +	 * 1. 1 byte "magic" identifier to allow format changes 
                  +	 * 
                  +	 * 2. 4 byte CRC32 of the payload 
                  +	 * 
                  +	 * 3. N - 5 byte payload 
                  +	 * 
                  +	 * If magic byte is 1 
                  +	 * 
                  +	 * 1. 1 byte "magic" identifier to allow format changes 
                  +	 * 
                  +	 * 2. 1 byte "attributes" identifier to allow annotations on the message independent of the version (e.g. compression enabled, type of codec used) 
                  +	 * 
                  +	 * 3. 4 byte CRC32 of the payload 
                  +	 * 
                  +	 * 4. N - 6 byte payload 
                  +	 * 
                  +	 */
                  +
                  +

                  +

                  5.5 Log

                  +

                  +A log for a topic named "my_topic" with two partitions consists of two directories (namely my_topic_0 and my_topic_1) populated with data files containing the messages for that topic. The format of the log files is a sequence of "log entries""; each log entry is a 4 byte integer N storing the message length which is followed by the N message bytes. Each message is uniquely identified by a 64-bit integer offset giving the byte position of the start of this message in the stream of all messages ever sent to that topic on that partition. The on-disk format of each message is given below. Each log file is named with the offset of the first message it contains. So the first file created will be 00000000000.kafka, and each additional file will have an integer name roughly S bytes from the previous file where S is the max log file size given in the configuration. +

                  +

                  +The exact binary format for messages is versioned and maintained as a standard interface so message sets can be transfered between producer, broker, and client without recopying or conversion when desirable. This format is as follows: +

                  +
                  +On-disk format of a message
                  +
                  +message length : 4 bytes (value: 1+4+n) 
                  +"magic" value  : 1 byte
                  +crc            : 4 bytes
                  +payload        : n bytes
                  +
                  +

                  +The use of the message offset as the message id is unusual. Our original idea was to use a GUID generated by the producer, and maintain a mapping from GUID to offset on each broker. But since a consumer must maintain an ID for each server, the global uniqueness of the GUID provides no value. Furthermore the complexity of maintaining the mapping from a random id to an offset requires a heavy weight index structure which must be synchronized with disk, essentially requiring a full persistent random-access data structure. Thus to simplify the lookup structure we decided to use a simple per-partition atomic counter which could be coupled with the partition id and node id to uniquely identify a message; this makes the lookup structure simpler, though multiple seeks per consumer request are still likely. However once we settled on a counter, the jump to directly using the offset seemed natural—both after all are monotonically increasing integers unique to a partition. Since the offset is hidden from the consumer API this decision is ultimately an implementation detail and we went with the more efficient approach. +

                  + +

                  Writes

                  +

                  +The log allows serial appends which always go to the last file. This file is rolled over to a fresh file when it reaches a configurable size (say 1GB). The log takes two configuration parameter M which gives the number of messages to write before forcing the OS to flush the file to disk, and S which gives a number of seconds after which a flush is forced. This gives a durability guarantee of losing at most M messages or S seconds of data in the event of a system crash. +

                  +

                  Reads

                  +

                  +Reads are done by giving the 64-bit logical offset of a message and an S-byte max chunk size. This will return an iterator over the messages contained in the S-byte buffer. S is intended to be larger than any single message, but in the event of an abnormally large message, the read can be retried multiple times, each time doubling the buffer size, until the message is read successfully. A maximum message and buffer size can be specified to make the server reject messages larger than some size, and to give a bound to the client on the maximum it need ever read to get a complete message. It is likely that the read buffer ends with a partial message, this is easily detected by the size delimiting. +

                  +

                  +The actual process of reading from an offset requires first locating the log segment file in which the data is stored, calculating the file-specific offset from the global offset value, and then reading from that file offset. The search is done as a simple binary search variation against an in-memory range maintained for each file. +

                  +

                  +The log provides the capability of getting the most recently written message to allow clients to start subscribing as of "right now". This is also useful in the case the consumer fails to consume its data within its SLA-specified number of days. In this case when the client attempts to consume a non-existant offset it is given an OutOfRangeException and can either reset itself or fail as appropriate to the use case. +

                  + +

                  The following is the format of the results sent to the consumer. + +

                  +MessageSetSend (fetch result)
                  +
                  +total length     : 4 bytes
                  +error code       : 2 bytes
                  +message 1        : x bytes
                  +...
                  +message n        : x bytes
                  +
                  + +
                  +MultiMessageSetSend (multiFetch result)
                  +
                  +total length       : 4 bytes
                  +error code         : 2 bytes
                  +messageSetSend 1
                  +...
                  +messageSetSend n
                  +
                  + +

                  Deletes

                  +

                  +Data is deleted one log segment at a time. The log manager allows pluggable delete policies to choose which files are eligible for deletion. The current policy deletes any log with a modification time of more than N days ago, though a policy which retained the last N GB could also be useful. To avoid locking reads while still allowing deletes that modify the segment list we use a copy-on-write style segment list implementation that provides consistent views to allow a binary search to proceed on an immutable static snapshot view of the log segments while deletes are progressing. +

                  +

                  Guarantees

                  +

                  +The log provides a configuration parameter M which controls the maximum number of messages that are written before forcing a flush to disk. On startup a log recovery process is run that iterates over all messages in the newest log segment and verifies that each message entry is valid. A message entry is valid if the sum of its size and offset are less than the length of the file AND the CRC32 of the message payload matches the CRC stored with the message. In the event corruption is detected the log is truncated to the last valid offset. +

                  +

                  +Note that two kinds of corruption must be handled: truncation in which an unwritten block is lost due to a crash, and corruption in which a nonsense block is ADDED to the file. The reason for this is that in general the OS makes no guarantee of the write order between the file inode and the actual block data so in addition to losing written data the file can gain nonsense data if the inode is updated with a new size but a crash occurs before the block containing that data is not written. The CRC detects this corner case, and prevents it from corrupting the log (though the unwritten messages are, of course, lost). +

                  + +

                  5.6 Distribution

                  +

                  Consumer Offset Tracking

                  +

                  +The high-level consumer tracks the maximum offset it has consumed in each partition and periodically commits its offset vector so that it can resume from those offsets in the event of a restart. Kafka provides the option to store all the offsets for a given consumer group in a designated broker (for that group) called the offset manager. i.e., any consumer instance in that consumer group should send its offset commits and fetches to that offset manager (broker). The high-level consumer handles this automatically. If you use the simple consumer you will need to manage offsets manually. This is currently unsupported in the Java simple consumer which can only commit or fetch offsets in ZooKeeper. If you use the Scala simple consumer you can discover the offset manager and explicitly commit or fetch offsets to the offset manager. A consumer can look up its offset manager by issuing a ConsumerMetadataRequest to any Kafka broker and reading the ConsumerMetadataResponse which will contain the offset manager. The consumer can then proceed to commit or fetch offsets from the offsets manager broker. In case the offset manager moves, the consumer will need to rediscover the offset manager. If you wish to manage your offsets manually, you can take a look at these code samples that explain how to issue OffsetCommitRequest and OffsetFetchRequest. +

                  + +

                  +When the offset manager receives an OffsetCommitRequest, it appends the request to a special compacted Kafka topic named __consumer_offsets. The offset manager sends a successful offset commit response to the consumer only after all the replicas of the offsets topic receive the offsets. In case the offsets fail to replicate within a configurable timeout, the offset commit will fail and the consumer may retry the commit after backing off. (This is done automatically by the high-level consumer.) The brokers periodically compact the offsets topic since it only needs to maintain the most recent offset commit per partition. The offset manager also caches the offsets in an in-memory table in order to serve offset fetches quickly. +

                  + +

                  +When the offset manager receives an offset fetch request, it simply returns the last committed offset vector from the offsets cache. In case the offset manager was just started or if it just became the offset manager for a new set of consumer groups (by becoming a leader for a partition of the offsets topic), it may need to load the offsets topic partition into the cache. In this case, the offset fetch will fail with an OffsetsLoadInProgress exception and the consumer may retry the OffsetFetchRequest after backing off. (This is done automatically by the high-level consumer.) +

                  + +
                  Migrating offsets from ZooKeeper to Kafka
                  +

                  +Kafka consumers in earlier releases store their offsets by default in ZooKeeper. It is possible to migrate these consumers to commit offsets into Kafka by following these steps: +

                    +
                  1. Set offsets.storage=kafka and dual.commit.enabled=true in your consumer config. +
                  2. +
                  3. Do a rolling bounce of your consumers and then verify that your consumers are healthy. +
                  4. +
                  5. Set dual.commit.enabled=false in your consumer config. +
                  6. +
                  7. Do a rolling bounce of your consumers and then verify that your consumers are healthy. +
                  8. +
                  +A roll-back (i.e., migrating from Kafka back to ZooKeeper) can also be performed using the above steps if you set offsets.storage=zookeeper. +

                  + +

                  ZooKeeper Directories

                  +

                  +The following gives the ZooKeeper structures and algorithms used for co-ordination between consumers and brokers. +

                  + +

                  Notation

                  +

                  +When an element in a path is denoted [xyz], that means that the value of xyz is not fixed and there is in fact a ZooKeeper znode for each possible value of xyz. For example /topics/[topic] would be a directory named /topics containing a sub-directory for each topic name. Numerical ranges are also given such as [0...5] to indicate the subdirectories 0, 1, 2, 3, 4. An arrow -> is used to indicate the contents of a znode. For example /hello -> world would indicate a znode /hello containing the value "world". +

                  + +

                  Broker Node Registry

                  +
                  +/brokers/ids/[0...N] --> host:port (ephemeral node)
                  +
                  +

                  +This is a list of all present broker nodes, each of which provides a unique logical broker id which identifies it to consumers (which must be given as part of its configuration). On startup, a broker node registers itself by creating a znode with the logical broker id under /brokers/ids. The purpose of the logical broker id is to allow a broker to be moved to a different physical machine without affecting consumers. An attempt to register a broker id that is already in use (say because two servers are configured with the same broker id) is an error. +

                  +

                  +Since the broker registers itself in ZooKeeper using ephemeral znodes, this registration is dynamic and will disappear if the broker is shutdown or dies (thus notifying consumers it is no longer available). +

                  +

                  Broker Topic Registry

                  +
                  +/brokers/topics/[topic]/[0...N] --> nPartions (ephemeral node)
                  +
                  + +

                  +Each broker registers itself under the topics it maintains and stores the number of partitions for that topic. +

                  + +

                  Consumers and Consumer Groups

                  +

                  +Consumers of topics also register themselves in ZooKeeper, in order to coordinate with each other and balance the consumption of data. Consumers can also store their offsets in ZooKeeper by setting offsets.storage=zookeeper. However, this offset storage mechanism will be deprecated in a future release. Therefore, it is recommended to migrate offsets storage to Kafka. +

                  + +

                  +Multiple consumers can form a group and jointly consume a single topic. Each consumer in the same group is given a shared group_id. +For example if one consumer is your foobar process, which is run across three machines, then you might assign this group of consumers the id "foobar". This group id is provided in the configuration of the consumer, and is your way to tell the consumer which group it belongs to. +

                  + +

                  +The consumers in a group divide up the partitions as fairly as possible, each partition is consumed by exactly one consumer in a consumer group. +

                  + +

                  Consumer Id Registry

                  +

                  +In addition to the group_id which is shared by all consumers in a group, each consumer is given a transient, unique consumer_id (of the form hostname:uuid) for identification purposes. Consumer ids are registered in the following directory. +

                  +/consumers/[group_id]/ids/[consumer_id] --> {"topic1": #streams, ..., "topicN": #streams} (ephemeral node)
                  +
                  +Each of the consumers in the group registers under its group and creates a znode with its consumer_id. The value of the znode contains a map of <topic, #streams>. This id is simply used to identify each of the consumers which is currently active within a group. This is an ephemeral node so it will disappear if the consumer process dies. +

                  + +

                  Consumer Offsets

                  +

                  +Consumers track the maximum offset they have consumed in each partition. This value is stored in a ZooKeeper directory if offsets.storage=zookeeper. This valued is stored in a ZooKeeper directory. +

                  +
                  +/consumers/[group_id]/offsets/[topic]/[broker_id-partition_id] --> offset_counter_value ((persistent node)
                  +
                  + +

                  Partition Owner registry

                  + +

                  +Each broker partition is consumed by a single consumer within a given consumer group. The consumer must establish its ownership of a given partition before any consumption can begin. To establish its ownership, a consumer writes its own id in an ephemeral node under the particular broker partition it is claiming. +

                  + +
                  +/consumers/[group_id]/owners/[topic]/[broker_id-partition_id] --> consumer_node_id (ephemeral node)
                  +
                  + +

                  Broker node registration

                  + +

                  +The broker nodes are basically independent, so they only publish information about what they have. When a broker joins, it registers itself under the broker node registry directory and writes information about its host name and port. The broker also register the list of existing topics and their logical partitions in the broker topic registry. New topics are registered dynamically when they are created on the broker. +

                  + +

                  Consumer registration algorithm

                  + +

                  +When a consumer starts, it does the following: +

                    +
                  1. Register itself in the consumer id registry under its group. +
                  2. +
                  3. Register a watch on changes (new consumers joining or any existing consumers leaving) under the consumer id registry. (Each change triggers rebalancing among all consumers within the group to which the changed consumer belongs.) +
                  4. +
                  5. Register a watch on changes (new brokers joining or any existing brokers leaving) under the broker id registry. (Each change triggers rebalancing among all consumers in all consumer groups.)
                  6. +
                  7. If the consumer creates a message stream using a topic filter, it also registers a watch on changes (new topics being added) under the broker topic registry. (Each change will trigger re-evaluation of the available topics to determine which topics are allowed by the topic filter. A new allowed topic will trigger rebalancing among all consumers within the consumer group.)
                  8. +
                  9. Force itself to rebalance within in its consumer group. +
                  10. +
                  +

                  + +

                  Consumer rebalancing algorithm

                  +

                  +The consumer rebalancing algorithms allows all the consumers in a group to come into consensus on which consumer is consuming which partitions. Consumer rebalancing is triggered on each addition or removal of both broker nodes and other consumers within the same group. For a given topic and a given consumer group, broker partitions are divided evenly among consumers within the group. A partition is always consumed by a single consumer. This design simplifies the implementation. Had we allowed a partition to be concurrently consumed by multiple consumers, there would be contention on the partition and some kind of locking would be required. If there are more consumers than partitions, some consumers won't get any data at all. During rebalancing, we try to assign partitions to consumers in such a way that reduces the number of broker nodes each consumer has to connect to. +

                  +

                  +Each consumer does the following during rebalancing: +

                  +
                  +   1. For each topic T that Ci subscribes to 
                  +   2.   let PT be all partitions producing topic T
                  +   3.   let CG be all consumers in the same group as Ci that consume topic T
                  +   4.   sort PT (so partitions on the same broker are clustered together)
                  +   5.   sort CG
                  +   6.   let i be the index position of Ci in CG and let N = size(PT)/size(CG)
                  +   7.   assign partitions from i*N to (i+1)*N - 1 to consumer Ci
                  +   8.   remove current entries owned by Ci from the partition owner registry
                  +   9.   add newly assigned partitions to the partition owner registry
                  +        (we may need to re-try this until the original partition owner releases its ownership)
                  +
                  +

                  +When rebalancing is triggered at one consumer, rebalancing should be triggered in other consumers within the same group about the same time. +

                  diff --git a/docs/introduction.html b/docs/introduction.html new file mode 100644 index 0000000000000..92a78265bd4b1 --- /dev/null +++ b/docs/introduction.html @@ -0,0 +1,99 @@ + + +

                  1.1 Introduction

                  +Kafka is a distributed, partitioned, replicated commit log service. It provides the functionality of a messaging system, but with a unique design. +

                  +What does all that mean? +

                  +First let's review some basic messaging terminology: +

                    +
                  • Kafka maintains feeds of messages in categories called topics. +
                  • We'll call processes that publish messages to a Kafka topic producers. +
                  • We'll call processes that subscribe to topics and process the feed of published messages consumers.. +
                  • Kafka is run as a cluster comprised of one or more servers each of which is called a broker. +
                  + +So, at a high level, producers send messages over the network to the Kafka cluster which in turn serves them up to consumers like this: +
                  + +
                  + +Communication between the clients and the servers is done with a simple, high-performance, language agnostic TCP protocol. We provide a Java client for Kafka, but clients are available in many languages. + +

                  Topics and Logs

                  +Let's first dive into the high-level abstraction Kafka provides—the topic. +

                  +A topic is a category or feed name to which messages are published. For each topic, the Kafka cluster maintains a partitioned log that looks like this: +

                  + +
                  +Each partition is an ordered, immutable sequence of messages that is continually appended to—a commit log. The messages in the partitions are each assigned a sequential id number called the offset that uniquely identifies each message within the partition. +

                  +The Kafka cluster retains all published messages—whether or not they have been consumed—for a configurable period of time. For example if the log retention is set to two days, then for the two days after a message is published it is available for consumption, after which it will be discarded to free up space. Kafka's performance is effectively constant with respect to data size so retaining lots of data is not a problem. +

                  +In fact the only metadata retained on a per-consumer basis is the position of the consumer in the log, called the "offset". This offset is controlled by the consumer: normally a consumer will advance its offset linearly as it reads messages, but in fact the position is controlled by the consumer and it can consume messages in any order it likes. For example a consumer can reset to an older offset to reprocess. +

                  +This combination of features means that Kafka consumers are very cheap—they can come and go without much impact on the cluster or on other consumers. For example, you can use our command line tools to "tail" the contents of any topic without changing what is consumed by any existing consumers. +

                  +The partitions in the log serve several purposes. First, they allow the log to scale beyond a size that will fit on a single server. Each individual partition must fit on the servers that host it, but a topic may have many partitions so it can handle an arbitrary amount of data. Second they act as the unit of parallelism—more on that in a bit. + +

                  Distribution

                  + +The partitions of the log are distributed over the servers in the Kafka cluster with each server handling data and requests for a share of the partitions. Each partition is replicated across a configurable number of servers for fault tolerance. +

                  +Each partition has one server which acts as the "leader" and zero or more servers which act as "followers". The leader handles all read and write requests for the partition while the followers passively replicate the leader. If the leader fails, one of the followers will automatically become the new leader. Each server acts as a leader for some of its partitions and a follower for others so load is well balanced within the cluster. + +

                  Producers

                  + +Producers publish data to the topics of their choice. The producer is responsible for choosing which message to assign to which partition within the topic. This can be done in a round-robin fashion simply to balance load or it can be done according to some semantic partition function (say based on some key in the message). More on the use of partitioning in a second. + +

                  Consumers

                  + +Messaging traditionally has two models: queuing and publish-subscribe. In a queue, a pool of consumers may read from a server and each message goes to one of them; in publish-subscribe the message is broadcast to all consumers. Kafka offers a single consumer abstraction that generalizes both of these—the consumer group. +

                  +Consumers label themselves with a consumer group name, and each message published to a topic is delivered to one consumer instance within each subscribing consumer group. Consumer instances can be in separate processes or on separate machines. +

                  +If all the consumer instances have the same consumer group, then this works just like a traditional queue balancing load over the consumers. +

                  +If all the consumer instances have different consumer groups, then this works like publish-subscribe and all messages are broadcast to all consumers. +

                  +More commonly, however, we have found that topics have a small number of consumer groups, one for each "logical subscriber". Each group is composed of many consumer instances for scalability and fault tolerance. This is nothing more than publish-subscribe semantics where the subscriber is cluster of consumers instead of a single process. +

                  + +

                  +
                  + A two server Kafka cluster hosting four partitions (P0-P3) with two consumer groups. Consumer group A has two consumer instances and group B has four. +
                  +

                  +Kafka has stronger ordering guarantees than a traditional messaging system, too. +

                  +A traditional queue retains messages in-order on the server, and if multiple consumers consume from the queue then the server hands out messages in the order they are stored. However, although the server hands out messages in order, the messages are delivered asynchronously to consumers, so they may arrive out of order on different consumers. This effectively means the ordering of the messages is lost in the presence of parallel consumption. Messaging systems often work around this by having a notion of "exclusive consumer" that allows only one process to consume from a queue, but of course this means that there is no parallelism in processing. +

                  +Kafka does it better. By having a notion of parallelism—the partition—within the topics, Kafka is able to provide both ordering guarantees and load balancing over a pool of consumer processes. This is achieved by assigning the partitions in the topic to the consumers in the consumer group so that each partition is consumed by exactly one consumer in the group. By doing this we ensure that the consumer is the only reader of that partition and consumes the data in order. Since there are many partitions this still balances the load over many consumer instances. Note however that there cannot be more consumer instances in a consumer group than partitions. +

                  +Kafka only provides a total order over messages within a partition, not between different partitions in a topic. Per-partition ordering combined with the ability to partition data by key is sufficient for most applications. However, if you require a total order over messages this can be achieved with a topic that has only one partition, though this will mean only one consumer process per consumer group. + +

                  Guarantees

                  + +At a high-level Kafka gives the following guarantees: +
                    +
                  • Messages sent by a producer to a particular topic partition will be appended in the order they are sent. That is, if a message M1 is sent by the same producer as a message M2, and M1 is sent first, then M1 will have a lower offset than M2 and appear earlier in the log. +
                  • A consumer instance sees messages in the order they are stored in the log. +
                  • For a topic with replication factor N, we will tolerate up to N-1 server failures without losing any messages committed to the log. +
                  +More details on these guarantees are given in the design section of the documentation. diff --git a/docs/migration.html b/docs/migration.html new file mode 100644 index 0000000000000..18ab6d40268d2 --- /dev/null +++ b/docs/migration.html @@ -0,0 +1,34 @@ + + + +

                  Migrating from 0.7.x to 0.8

                  + +0.8 is our first (and hopefully last) release with a non-backwards-compatible wire protocol, ZooKeeper layout, and on-disk data format. This was a chance for us to clean up a lot of cruft and start fresh. This means performing a no-downtime upgrade is more painful than normal—you cannot just swap in the new code in-place. + +

                  Migration Steps

                  + +
                    +
                  1. Setup a new cluster running 0.8. +
                  2. Use the 0.7 to 0.8 migration tool to mirror data from the 0.7 cluster into the 0.8 cluster. +
                  3. When the 0.8 cluster is fully caught up, redeploy all data consumers running the 0.8 client and reading from the 0.8 cluster. +
                  4. Finally migrate all 0.7 producers to 0.8 client publishing data to the 0.8 cluster. +
                  5. Decomission the 0.7 cluster. +
                  6. Drink. +
                  + + diff --git a/docs/ops.html b/docs/ops.html new file mode 100644 index 0000000000000..2164ab7e31c31 --- /dev/null +++ b/docs/ops.html @@ -0,0 +1,876 @@ + + +Here is some information on actually running Kafka as a production system based on usage and experience at LinkedIn. Please send us any additional tips you know of. + +

                  6.1 Basic Kafka Operations

                  + +This section will review the most common operations you will perform on your Kafka cluster. All of the tools reviewed in this section are available under the bin/ directory of the Kafka distribution and each tool will print details on all possible commandline options if it is run with no arguments. + +

                  Adding and removing topics

                  + +You have the option of either adding topics manually or having them be created automatically when data is first published to a non-existent topic. If topics are auto-created then you may want to tune the default topic configurations used for auto-created topics. +

                  +Topics are added and modified using the topic tool: +

                  + > bin/kafka-topics.sh --zookeeper zk_host:port/chroot --create --topic my_topic_name 
                  +       --partitions 20 --replication-factor 3 --config x=y
                  +
                  +The replication factor controls how many servers will replicate each message that is written. If you have a replication factor of 3 then up to 2 servers can fail before you will lose access to your data. We recommend you use a replication factor of 2 or 3 so that you can transparently bounce machines without interrupting data consumption. +

                  +The partition count controls how many logs the topic will be sharded into. There are several impacts of the partition count. First each partition must fit entirely on a single server. So if you have 20 partitions the full data set (and read and write load) will be handled by no more than 20 servers (no counting replicas). Finally the partition count impacts the maximum parallelism of your consumers. This is discussed in greater detail in the concepts section. +

                  +The configurations added on the command line override the default settings the server has for things like the length of time data should be retained. The complete set of per-topic configurations is documented here. + +

                  Modifying topics

                  + +You can change the configuration or partitioning of a topic using the same topic tool. +

                  +To add partitions you can do +

                  + > bin/kafka-topics.sh --zookeeper zk_host:port/chroot --alter --topic my_topic_name 
                  +       --partitions 40 
                  +
                  +Be aware that one use case for partitions is to semantically partition data, and adding partitions doesn't change the partitioning of existing data so this may disturb consumers if they rely on that partition. That is if data is partitioned by hash(key) % number_of_partitions then this partitioning will potentially be shuffled by adding partitions but Kafka will not attempt to automatically redistribute data in any way. +

                  +To add configs: +

                  + > bin/kafka-topics.sh --zookeeper zk_host:port/chroot --alter --topic my_topic_name --config x=y
                  +
                  +To remove a config: +
                  + > bin/kafka-topics.sh --zookeeper zk_host:port/chroot --alter --topic my_topic_name --deleteConfig x
                  +
                  +And finally deleting a topic: +
                  + > bin/kafka-topics.sh --zookeeper zk_host:port/chroot --delete --topic my_topic_name
                  +
                  +Topic deletion option is disabled by default. To enable it set the server config +
                  delete.topic.enable=true
                  +

                  +Kafka does not currently support reducing the number of partitions for a topic or changing the replication factor. + +

                  Graceful shutdown

                  + +The Kafka cluster will automatically detect any broker shutdown or failure and elect new leaders for the partitions on that machine. This will occur whether a server fails or it is brought down intentionally for maintenance or configuration changes. For the later cases Kafka supports a more graceful mechanism for stoping a server then just killing it. + +When a server is stopped gracefully it has two optimizations it will take advantage of: +
                    +
                  1. It will sync all its logs to disk to avoid needing to do any log recovery when it restarts (i.e. validating the checksum for all messages in the tail of the log). Log recovery takes time so this speeds up intentional restarts. +
                  2. It will migrate any partitions the server is the leader for to other replicas prior to shutting down. This will make the leadership transfer faster and minimize the time each partition is unavailable to a few milliseconds. +
                  + +Syncing the logs will happen automatically happen whenever the server is stopped other than by a hard kill, but the controlled leadership migration requires using a special setting: +
                  +    controlled.shutdown.enable=true
                  +
                  +Note that controlled shutdown will only succeed if all the partitions hosted on the broker have replicas (i.e. the replication factor is greater than 1 and at least one of these replicas is alive). This is generally what you want since shutting down the last replica would make that topic partition unavailable. + +

                  Balancing leadership

                  + +Whenever a broker stops or crashes leadership for that broker's partitions transfers to other replicas. This means that by default when the broker is restarted it will only be a follower for all its partitions, meaning it will not be used for client reads and writes. +

                  +To avoid this imbalance, Kafka has a notion of preferred replicas. If the list of replicas for a partition is 1,5,9 then node 1 is preferred as the leader to either node 5 or 9 because it is earlier in the replica list. You can have the Kafka cluster try to restore leadership to the restored replicas by running the command: +

                  + > bin/kafka-preferred-replica-election.sh --zookeeper zk_host:port/chroot
                  +
                  + +Since running this command can be tedious you can also configure Kafka to do this automatically by setting the following configuration: +
                  +    auto.leader.rebalance.enable=true
                  +
                  + +

                  Mirroring data between clusters

                  + +We refer to the process of replicating data between Kafka clusters "mirroring" to avoid confusion with the replication that happens amongst the nodes in a single cluster. Kafka comes with a tool for mirroring data between Kafka clusters. The tool reads from one or more source clusters and writes to a destination cluster, like this: +

                  + +

                  +A common use case for this kind of mirroring is to provide a replica in another datacenter. This scenario will be discussed in more detail in the next section. +

                  +You can run many such mirroring processes to increase throughput and for fault-tolerance (if one process dies, the others will take overs the additional load). +

                  +Data will be read from topics in the source cluster and written to a topic with the same name in the destination cluster. In fact the mirror maker is little more than a Kafka consumer and producer hooked together. +

                  +The source and destination clusters are completely independent entities: they can have different numbers of partitions and the offsets will not be the same. For this reason the mirror cluster is not really intended as a fault-tolerance mechanism (as the consumer position will be different); for that we recommend using normal in-cluster replication. The mirror maker process will, however, retain and use the message key for partitioning so order is preserved on a per-key basis. +

                  +Here is an example showing how to mirror a single topic (named my-topic) from two input clusters: +

                  + > bin/kafka-run-class.sh kafka.tools.MirrorMaker
                  +       --consumer.config consumer-1.properties --consumer.config consumer-2.properties 
                  +       --producer.config producer.properties --whitelist my-topic
                  +
                  +Note that we specify the list of topics with the --whitelist option. This option allows any regular expression using Java-style regular expressions. So you could mirror two topics named A and B using --whitelist 'A|B'. Or you could mirror all topics using --whitelist '*'. Make sure to quote any regular expression to ensure the shell doesn't try to expand it as a file path. For convenience we allow the use of ',' instead of '|' to specify a list of topics. +

                  +Sometime it is easier to say what it is that you don't want. Instead of using --whitelist to say what you want to mirror you can use --blacklist to say what to exclude. This also takes a regular expression argument. +

                  +Combining mirroring with the configuration auto.create.topics.enable=true makes it possible to have a replica cluster that will automatically create and replicate all data in a source cluster even as new topics are added. + +

                  Checking consumer position

                  +Sometimes it's useful to see the position of your consumers. We have a tool that will show the position of all consumers in a consumer group as well as how far behind the end of the log they are. To run this tool on a consumer group named my-group consuming a topic named my-topic would look like this: +
                  + > bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect localhost:2181 --group test
                  +Group           Topic                          Pid Offset          logSize         Lag             Owner
                  +my-group        my-topic                       0   0               0               0               test_jkreps-mn-1394154511599-60744496-0
                  +my-group        my-topic                       1   0               0               0               test_jkreps-mn-1394154521217-1a0be913-0
                  +
                  + +

                  Expanding your cluster

                  + +Adding servers to a Kafka cluster is easy, just assign them a unique broker id and start up Kafka on your new servers. However these new servers will not automatically be assigned any data partitions, so unless partitions are moved to them they won't be doing any work until new topics are created. So usually when you add machines to your cluster you will want to migrate some existing data to these machines. +

                  +The process of migrating data is manually initiated but fully automated. Under the covers what happens is that Kafka will add the new server as a follower of the partition it is migrating and allow it to fully replicate the existing data in that partition. When the new server has fully replicated the contents of this partition and joined the in-sync replica one of the existing replicas will delete their partition's data. +

                  +The partition reassignment tool can be used to move partitions across brokers. An ideal partition distribution would ensure even data load and partition sizes across all brokers. In 0.8.1, the partition reassignment tool does not have the capability to automatically study the data distribution in a Kafka cluster and move partitions around to attain an even load distribution. As such, the admin has to figure out which topics or partitions should be moved around. +

                  +The partition reassignment tool can run in 3 mutually exclusive modes - +

                    +
                  • --generate: In this mode, given a list of topics and a list of brokers, the tool generates a candidate reassignment to move all partitions of the specified topics to the new brokers. This option merely provides a convenient way to generate a partition reassignment plan given a list of topics and target brokers.
                  • +
                  • --execute: In this mode, the tool kicks off the reassignment of partitions based on the user provided reassignment plan. (using the --reassignment-json-file option). This can either be a custom reassignment plan hand crafted by the admin or provided by using the --generate option
                  • +
                  • --verify: In this mode, the tool verifies the status of the reassignment for all partitions listed during the last --execute. The status can be either of successfully completed, failed or in progress
                  • +
                  +
                  Automatically migrating data to new machines
                  +The partition reassignment tool can be used to move some topics off of the current set of brokers to the newly added brokers. This is typically useful while expanding an existing cluster since it is easier to move entire topics to the new set of brokers, than moving one partition at a time. When used to do this, the user should provide a list of topics that should be moved to the new set of brokers and a target list of new brokers. The tool then evenly distributes all partitions for the given list of topics across the new set of brokers. During this move, the replication factor of the topic is kept constant. Effectively the replicas for all partitions for the input list of topics are moved from the old set of brokers to the newly added brokers. +

                  +For instance, the following example will move all partitions for topics foo1,foo2 to the new set of brokers 5,6. At the end of this move, all partitions for topics foo1 and foo2 will only exist on brokers 5,6 +

                  +Since, the tool accepts the input list of topics as a json file, you first need to identify the topics you want to move and create the json file as follows- +

                  +> cat topics-to-move.json
                  +{"topics": [{"topic": "foo1"},
                  +            {"topic": "foo2"}],
                  + "version":1
                  +}
                  +
                  +Once the json file is ready, use the partition reassignment tool to generate a candidate assignment- +
                  +> bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --topics-to-move-json-file topics-to-move.json --broker-list "5,6" --generate 
                  +Current partition replica assignment
                  +
                  +{"version":1,
                  + "partitions":[{"topic":"foo1","partition":2,"replicas":[1,2]},
                  +               {"topic":"foo1","partition":0,"replicas":[3,4]},
                  +               {"topic":"foo2","partition":2,"replicas":[1,2]},
                  +               {"topic":"foo2","partition":0,"replicas":[3,4]},
                  +               {"topic":"foo1","partition":1,"replicas":[2,3]},
                  +               {"topic":"foo2","partition":1,"replicas":[2,3]}]
                  +}
                  +
                  +Proposed partition reassignment configuration
                  +
                  +{"version":1,
                  + "partitions":[{"topic":"foo1","partition":2,"replicas":[5,6]},
                  +               {"topic":"foo1","partition":0,"replicas":[5,6]},
                  +               {"topic":"foo2","partition":2,"replicas":[5,6]},
                  +               {"topic":"foo2","partition":0,"replicas":[5,6]},
                  +               {"topic":"foo1","partition":1,"replicas":[5,6]},
                  +               {"topic":"foo2","partition":1,"replicas":[5,6]}]
                  +}
                  +
                  +

                  +The tool generates a candidate assignment that will move all partitions from topics foo1,foo2 to brokers 5,6. Note, however, that at this point, the partition movement has not started, it merely tells you the current assignment and the proposed new assignment. The current assignment should be saved in case you want to rollback to it. The new assignment should be saved in a json file (e.g. expand-cluster-reassignment.json) to be input to the tool with the --execute option as follows- +

                  +> bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --reassignment-json-file expand-cluster-reassignment.json --execute
                  +Current partition replica assignment
                  +
                  +{"version":1,
                  + "partitions":[{"topic":"foo1","partition":2,"replicas":[1,2]},
                  +               {"topic":"foo1","partition":0,"replicas":[3,4]},
                  +               {"topic":"foo2","partition":2,"replicas":[1,2]},
                  +               {"topic":"foo2","partition":0,"replicas":[3,4]},
                  +               {"topic":"foo1","partition":1,"replicas":[2,3]},
                  +               {"topic":"foo2","partition":1,"replicas":[2,3]}]
                  +}
                  +
                  +Save this to use as the --reassignment-json-file option during rollback
                  +Successfully started reassignment of partitions
                  +{"version":1,
                  + "partitions":[{"topic":"foo1","partition":2,"replicas":[5,6]},
                  +               {"topic":"foo1","partition":0,"replicas":[5,6]},
                  +               {"topic":"foo2","partition":2,"replicas":[5,6]},
                  +               {"topic":"foo2","partition":0,"replicas":[5,6]},
                  +               {"topic":"foo1","partition":1,"replicas":[5,6]},
                  +               {"topic":"foo2","partition":1,"replicas":[5,6]}]
                  +}
                  +
                  +

                  +Finally, the --verify option can be used with the tool to check the status of the partition reassignment. Note that the same expand-cluster-reassignment.json (used with the --execute option) should be used with the --verify option +

                  +> bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --reassignment-json-file expand-cluster-reassignment.json --verify
                  +Status of partition reassignment:
                  +Reassignment of partition [foo1,0] completed successfully
                  +Reassignment of partition [foo1,1] is in progress
                  +Reassignment of partition [foo1,2] is in progress
                  +Reassignment of partition [foo2,0] completed successfully
                  +Reassignment of partition [foo2,1] completed successfully 
                  +Reassignment of partition [foo2,2] completed successfully 
                  +
                  + +
                  Custom partition assignment and migration
                  +The partition reassignment tool can also be used to selectively move replicas of a partition to a specific set of brokers. When used in this manner, it is assumed that the user knows the reassignment plan and does not require the tool to generate a candidate reassignment, effectively skipping the --generate step and moving straight to the --execute step +

                  +For instance, the following example moves partition 0 of topic foo1 to brokers 5,6 and partition 1 of topic foo2 to brokers 2,3 +

                  +The first step is to hand craft the custom reassignment plan in a json file- +

                  +> cat custom-reassignment.json
                  +{"version":1,"partitions":[{"topic":"foo1","partition":0,"replicas":[5,6]},{"topic":"foo2","partition":1,"replicas":[2,3]}]}
                  +
                  +Then, use the json file with the --execute option to start the reassignment process- +
                  +> bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --reassignment-json-file custom-reassignment.json --execute
                  +Current partition replica assignment
                  +
                  +{"version":1,
                  + "partitions":[{"topic":"foo1","partition":0,"replicas":[1,2]},
                  +               {"topic":"foo2","partition":1,"replicas":[3,4]}]
                  +}
                  +
                  +Save this to use as the --reassignment-json-file option during rollback
                  +Successfully started reassignment of partitions
                  +{"version":1,
                  + "partitions":[{"topic":"foo1","partition":0,"replicas":[5,6]},
                  +               {"topic":"foo2","partition":1,"replicas":[2,3]}]
                  +}
                  +
                  +

                  +The --verify option can be used with the tool to check the status of the partition reassignment. Note that the same expand-cluster-reassignment.json (used with the --execute option) should be used with the --verify option +

                  +bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --reassignment-json-file custom-reassignment.json --verify
                  +Status of partition reassignment:
                  +Reassignment of partition [foo1,0] completed successfully
                  +Reassignment of partition [foo2,1] completed successfully 
                  +
                  + +

                  Decommissioning brokers

                  +The partition reassignment tool does not have the ability to automatically generate a reassignment plan for decommissioning brokers yet. As such, the admin has to come up with a reassignment plan to move the replica for all partitions hosted on the broker to be decommissioned, to the rest of the brokers. This can be relatively tedious as the reassignment needs to ensure that all the replicas are not moved from the decommissioned broker to only one other broker. To make this process effortless, we plan to add tooling support for decommissioning brokers in 0.8.2. + +

                  Increasing replication factor

                  +Increasing the replication factor of an existing partition is easy. Just specify the extra replicas in the custom reassignment json file and use it with the --execute option to increase the replication factor of the specified partitions. +

                  +For instance, the following example increases the replication factor of partition 0 of topic foo from 1 to 3. Before increasing the replication factor, the partition's only replica existed on broker 5. As part of increasing the replication factor, we will add more replicas on brokers 6 and 7. +

                  +The first step is to hand craft the custom reassignment plan in a json file- +

                  +> cat increase-replication-factor.json
                  +{"version":1,
                  + "partitions":[{"topic":"foo","partition":0,"replicas":[5,6,7]}]}
                  +
                  +Then, use the json file with the --execute option to start the reassignment process- +
                  +> bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --reassignment-json-file increase-replication-factor.json --execute
                  +Current partition replica assignment
                  +
                  +{"version":1,
                  + "partitions":[{"topic":"foo","partition":0,"replicas":[5]}]}
                  +
                  +Save this to use as the --reassignment-json-file option during rollback
                  +Successfully started reassignment of partitions
                  +{"version":1,
                  + "partitions":[{"topic":"foo","partition":0,"replicas":[5,6,7]}]}
                  +
                  +

                  +The --verify option can be used with the tool to check the status of the partition reassignment. Note that the same increase-replication-factor.json (used with the --execute option) should be used with the --verify option +

                  +bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --reassignment-json-file increase-replication-factor.json --verify
                  +Status of partition reassignment:
                  +Reassignment of partition [foo,0] completed successfully
                  +
                  +You can also verify the increase in replication factor with the kafka-topics tool- +
                  +> bin/kafka-topics.sh --zookeeper localhost:2181 --topic foo --describe
                  +Topic:foo	PartitionCount:1	ReplicationFactor:3	Configs:
                  +	Topic: foo	Partition: 0	Leader: 5	Replicas: 5,6,7	Isr: 5,6,7
                  +
                  + +

                  6.2 Datacenters

                  + +Some deployments will need to manage a data pipeline that spans multiple datacenters. Our recommended approach to this is to deploy a local Kafka cluster in each datacenter with application instances in each datacenter interacting only with their local cluster and mirroring between clusters (see the documentation on the mirror maker tool for how to do this). +

                  +This deployment pattern allows datacenters to act as independent entities and allows us to manage and tune inter-datacenter replication centrally. This allows each facility to stand alone and operate even if the inter-datacenter links are unavailable: when this occurs the mirroring falls behind until the link is restored at which time it catches up. +

                  +For applications that need a global view of all data you can use mirroring to provide clusters which have aggregate data mirrored from the local clusters in all datacenters. These aggregate clusters are used for reads by applications that require the full data set. +

                  +This is not the only possible deployment pattern. It is possible to read from or write to a remote Kafka cluster over the WAN, though obviously this will add whatever latency is required to get the cluster. +

                  +Kafka naturally batches data in both the producer and consumer so it can achieve high-throughput even over a high-latency connection. To allow this though it may be necessary to increase the TCP socket buffer sizes for the producer, consumer, and broker using the socket.send.buffer.bytes and socket.receive.buffer.bytes configurations. The appropriate way to set this is documented here. +

                  +It is generally not advisable to run a single Kafka cluster that spans multiple datacenters over a high-latency link. This will incur very high replication latency both for Kafka writes and ZooKeeper writes, and neither Kafka nor ZooKeeper will remain available in all locations if the network between locations is unavailable. + +

                  6.3 Kafka Configuration

                  + +

                  Important Client Configurations

                  +The most important producer configurations control +
                    +
                  • compression
                  • +
                  • sync vs async production
                  • +
                  • batch size (for async producers)
                  • +
                  +The most important consumer configuration is the fetch size. +

                  +All configurations are documented in the configuration section. +

                  +

                  A Production Server Config

                  +Here is our server production server configuration: +
                  +# Replication configurations
                  +num.replica.fetchers=4
                  +replica.fetch.max.bytes=1048576
                  +replica.fetch.wait.max.ms=500
                  +replica.high.watermark.checkpoint.interval.ms=5000
                  +replica.socket.timeout.ms=30000
                  +replica.socket.receive.buffer.bytes=65536
                  +replica.lag.time.max.ms=10000
                  +
                  +controller.socket.timeout.ms=30000
                  +controller.message.queue.size=10
                  +
                  +# Log configuration
                  +num.partitions=8
                  +message.max.bytes=1000000
                  +auto.create.topics.enable=true
                  +log.index.interval.bytes=4096
                  +log.index.size.max.bytes=10485760
                  +log.retention.hours=168
                  +log.flush.interval.ms=10000
                  +log.flush.interval.messages=20000
                  +log.flush.scheduler.interval.ms=2000
                  +log.roll.hours=168
                  +log.retention.check.interval.ms=300000
                  +log.segment.bytes=1073741824
                  +
                  +# ZK configuration
                  +zookeeper.connection.timeout.ms=6000
                  +zookeeper.sync.time.ms=2000
                  +
                  +# Socket server configuration
                  +num.io.threads=8
                  +num.network.threads=8
                  +socket.request.max.bytes=104857600
                  +socket.receive.buffer.bytes=1048576
                  +socket.send.buffer.bytes=1048576
                  +queued.max.requests=16
                  +fetch.purgatory.purge.interval.requests=100
                  +producer.purgatory.purge.interval.requests=100
                  +
                  + +Our client configuration varies a fair amount between different use cases. + +

                  Java Version

                  +We're currently running JDK 1.7 u51, and we've switched over to the G1 collector. If you do this (and we highly recommend it), make sure you're on u51. We tried out u21 in testing, but we had a number of problems with the GC implementation in that version. + +Our tuning looks like this: +
                  +-Xms4g -Xmx4g -XX:PermSize=48m -XX:MaxPermSize=48m -XX:+UseG1GC
                  +-XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35
                  +
                  + +For reference, here are the stats on one of LinkedIn's busiest clusters (at peak): + - 15 brokers + - 15.5k partitions (replication factor 2) + - 400k messages/sec in + - 70 MB/sec inbound, 400 MB/sec+ outbound + +The tuning looks fairly aggressive, but all of the brokers in that cluster have a 90% GC pause time of about 21ms, and they're doing less than 1 young GC per second. + +

                  6.4 Hardware and OS

                  +We are using dual quad-core Intel Xeon machines with 24GB of memory. +

                  +You need sufficient memory to buffer active readers and writers. You can do a back-of-the-envelope estimate of memory needs by assuming you want to be able to buffer for 30 seconds and compute your memory need as write_throughput*30. +

                  +The disk throughput is important. We have 8x7200 rpm SATA drives. In general disk throughput is the performance bottleneck, and more disks is more better. Depending on how you configure flush behavior you may or may not benefit from more expensive disks (if you force flush often then higher RPM SAS drives may be better). + +

                  OS

                  +Kafka should run well on any unix system and has been tested on Linux and Solaris. +

                  +We have seen a few issues running on Windows and Windows is not currently a well supported platform though we would be happy to change that. +

                  +You likely don't need to do much OS-level tuning though there are a few things that will help performance. +

                  +Two configurations that may be important: +

                    +
                  • We upped the number of file descriptors since we have lots of topics and lots of connections. +
                  • We upped the max socket buffer size to enable high-performance data transfer between data centers described here. +
                  + +

                  Disks and Filesystem

                  +We recommend using multiple drives to get good throughput and not sharing the same drives used for Kafka data with application logs or other OS filesystem activity to ensure good latency. As of 0.8 you can either RAID these drives together into a single volume or format and mount each drive as its own directory. Since Kafka has replication the redundancy provided by RAID can also be provided at the application level. This choice has several tradeoffs. +

                  +If you configure multiple data directories partitions will be assigned round-robin to data directories. Each partition will be entirely in one of the data directories. If data is not well balanced among partitions this can lead to load imbalance between disks. +

                  +RAID can potentially do better at balancing load between disks (although it doesn't always seem to) because it balances load at a lower level. The primary downside of RAID is that it is usually a big performance hit for write throughput and reduces the available disk space. +

                  +Another potential benefit of RAID is the ability to tolerate disk failures. However our experience has been that rebuilding the RAID array is so I/O intensive that it effectively disables the server, so this does not provide much real availability improvement. + +

                  Application vs. OS Flush Management

                  +Kafka always immediately writes all data to the filesystem and supports the ability to configure the flush policy that controls when data is forced out of the OS cache and onto disk using the and flush. This flush policy can be controlled to force data to disk after a period of time or after a certain number of messages has been written. There are several choices in this configuration. +

                  +Kafka must eventually call fsync to know that data was flushed. When recovering from a crash for any log segment not known to be fsync'd Kafka will check the integrity of each message by checking its CRC and also rebuild the accompanying offset index file as part of the recovery process executed on startup. +

                  +Note that durability in Kafka does not require syncing data to disk, as a failed node will always recover from its replicas. +

                  +We recommend using the default flush settings which disable application fsync entirely. This means relying on the background flush done by the OS and Kafka's own background flush. This provides the best of all worlds for most uses: no knobs to tune, great throughput and latency, and full recovery guarantees. We generally feel that the guarantees provided by replication are stronger than sync to local disk, however the paranoid still may prefer having both and application level fsync policies are still supported. +

                  +The drawback of using application level flush settings are that this is less efficient in it's disk usage pattern (it gives the OS less leeway to re-order writes) and it can introduce latency as fsync in most Linux filesystems blocks writes to the file whereas the background flushing does much more granular page-level locking. +

                  +In general you don't need to do any low-level tuning of the filesystem, but in the next few sections we will go over some of this in case it is useful. + +

                  Understanding Linux OS Flush Behavior

                  + +In Linux, data written to the filesystem is maintained in pagecache until it must be written out to disk (due to an application-level fsync or the OS's own flush policy). The flushing of data is done by a set of background threads called pdflush (or in post 2.6.32 kernels "flusher threads"). +

                  +Pdflush has a configurable policy that controls how much dirty data can be maintained in cache and for how long before it must be written back to disk. This policy is described here. When Pdflush cannot keep up with the rate of data being written it will eventually cause the writing process to block incurring latency in the writes to slow down the accumulation of data. +

                  +You can see the current state of OS memory usage by doing +

                  +  > cat /proc/meminfo
                  +
                  +The meaning of these values are described in the link above. +

                  +Using pagecache has several advantages over an in-process cache for storing data that will be written out to disk: +

                    +
                  • The I/O scheduler will batch together consecutive small writes into bigger physical writes which improves throughput. +
                  • The I/O scheduler will attempt to re-sequence writes to minimize movement of the disk head which improves throughput. +
                  • It automatically uses all the free memory on the machine +
                  + +

                  Ext4 Notes

                  +Ext4 may or may not be the best filesystem for Kafka. Filesystems like XFS supposedly handle locking during fsync better. We have only tried Ext4, though. +

                  +It is not necessary to tune these settings, however those wanting to optimize performance have a few knobs that will help: +

                    +
                  • data=writeback: Ext4 defaults to data=ordered which puts a strong order on some writes. Kafka does not require this ordering as it does very paranoid data recovery on all unflushed log. This setting removes the ordering constraint and seems to significantly reduce latency. +
                  • Disabling journaling: Journaling is a tradeoff: it makes reboots faster after server crashes but it introduces a great deal of additional locking which adds variance to write performance. Those who don't care about reboot time and want to reduce a major source of write latency spikes can turn off journaling entirely. +
                  • commit=num_secs: This tunes the frequency with which ext4 commits to its metadata journal. Setting this to a lower value reduces the loss of unflushed data during a crash. Setting this to a higher value will improve throughput. +
                  • nobh: This setting controls additional ordering guarantees when using data=writeback mode. This should be safe with Kafka as we do not depend on write ordering and improves throughput and latency. +
                  • delalloc: Delayed allocation means that the filesystem avoid allocating any blocks until the physical write occurs. This allows ext4 to allocate a large extent instead of smaller pages and helps ensure the data is written sequentially. This feature is great for throughput. It does seem to involve some locking in the filesystem which adds a bit of latency variance. +
                  + +

                  6.6 Monitoring

                  + +Kafka uses Yammer Metrics for metrics reporting in both the server and the client. This can be configured to report stats using pluggable stats reporters to hook up to your monitoring system. +

                  +The easiest way to see the available metrics to fire up jconsole and point it at a running kafka client or server; this will all browsing all metrics with JMX. +

                  +We pay particular we do graphing and alerting on the following metrics: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
                  DescriptionMbean nameNormal value
                  Message in ratekafka.server:type=BrokerTopicMetrics,name=MessagesInPerSec
                  Byte in ratekafka.server:type=BrokerTopicMetrics,name=BytesInPerSec
                  Request ratekafka.network:type=RequestMetrics,name=RequestsPerSec,request={Produce|FetchConsumer|FetchFollower}
                  Byte out ratekafka.server:type=BrokerTopicMetrics,name=BytesOutPerSec
                  Log flush rate and timekafka.log:type=LogFlushStats,name=LogFlushRateAndTimeMs
                  # of under replicated partitions (|ISR| < |all replicas|)kafka.server:type=ReplicaManager,name=UnderReplicatedPartitions0
                  Is controller active on brokerkafka.controller:type=KafkaController,name=ActiveControllerCountonly one broker in the cluster should have 1
                  Leader election ratekafka.controller:type=ControllerStats,name=LeaderElectionRateAndTimeMsnon-zero when there are broker failures
                  Unclean leader election ratekafka.controller:type=ControllerStats,name=UncleanLeaderElectionsPerSec0
                  Partition countskafka.server:type=ReplicaManager,name=PartitionCountmostly even across brokers
                  Leader replica countskafka.server:type=ReplicaManager,name=LeaderCountmostly even across brokers
                  ISR shrink ratekafka.server:type=ReplicaManager,name=IsrShrinksPerSecIf a broker goes down, ISR for some of the partitions will + shrink. When that broker is up again, ISR will be expanded + once the replicas are fully caught up. Other than that, the + expected value for both ISR shrink rate and expansion rate is 0.
                  ISR expansion ratekafka.server:type=ReplicaManager,name=IsrExpandsPerSecSee above
                  Max lag in messages btw follower and leader replicaskafka.server:type=ReplicaFetcherManager,name=MaxLag,clientId=Replicalag should be proportional to the maximum batch size of a produce request.
                  Lag in messages per follower replicakafka.server:type=FetcherLagMetrics,name=ConsumerLag,clientId=([-.\w]+),topic=([-.\w]+),partition=([0-9]+)lag should be proportional to the maximum batch size of a produce request.
                  Requests waiting in the producer purgatorykafka.server:type=ProducerRequestPurgatory,name=PurgatorySizenon-zero if ack=-1 is used
                  Requests waiting in the fetch purgatorykafka.server:type=FetchRequestPurgatory,name=PurgatorySizesize depends on fetch.wait.max.ms in the consumer
                  Request total timekafka.network:type=RequestMetrics,name=TotalTimeMs,request={Produce|FetchConsumer|FetchFollower}broken into queue, local, remote and response send time
                  Time the request waiting in the request queuekafka.network:type=RequestMetrics,name=QueueTimeMs,request={Produce|FetchConsumer|FetchFollower}
                  Time the request being processed at the leaderkafka.network:type=RequestMetrics,name=LocalTimeMs,request={Produce|FetchConsumer|FetchFollower}
                  Time the request waits for the followerkafka.network:type=RequestMetrics,name=RemoteTimeMs,request={Produce|FetchConsumer|FetchFollower}non-zero for produce requests when ack=-1
                  Time to send the responsekafka.network:type=RequestMetrics,name=ResponseSendTimeMs,request={Produce|FetchConsumer|FetchFollower}
                  Number of messages the consumer lags behind the producer bykafka.consumer:type=ConsumerFetcherManager,name=MaxLag,clientId=([-.\w]+)
                  The average fraction of time the network processors are idlekafka.network:type=SocketServer,name=NetworkProcessorAvgIdlePercentbetween 0 and 1, ideally > 0.3
                  The average fraction of time the request handler threads are idlekafka.server:type=KafkaRequestHandlerPool,name=RequestHandlerAvgIdlePercentbetween 0 and 1, ideally > 0.3
                  + +

                  New producer monitoring

                  + +The following metrics are available on new producer instances. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
                  Metric/Attribute nameDescriptionMbean name
                  waiting-threadsThe number of user threads blocked waiting for buffer memory to enqueue their recordskafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  buffer-total-bytesThe maximum amount of buffer memory the client can use (whether or not it is currently used).kafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  buffer-available-bytesThe total amount of buffer memory that is not being used (either unallocated or in the free list).kafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  bufferpool-wait-timeThe fraction of time an appender waits for space allocation.kafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  batch-size-avgThe average number of bytes sent per partition per-request.kafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  batch-size-maxThe max number of bytes sent per partition per-request.kafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  compression-rate-avgThe average compression rate of record batches.kafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  record-queue-time-avgThe average time in ms record batches spent in the record accumulator.kafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  record-queue-time-maxThe maximum time in ms record batches spent in the record accumulatorkafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  request-latency-avgThe average request latency in mskafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  request-latency-maxThe maximum request latency in mskafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  record-send-rateThe average number of records sent per second.kafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  records-per-request-avgThe average number of records per request.kafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  record-retry-rateThe average per-second number of retried record sendskafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  record-error-rateThe average per-second number of record sends that resulted in errorskafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  record-size-maxThe maximum record sizekafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  record-size-avgThe average record sizekafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  requests-in-flightThe current number of in-flight requests awaiting a response.kafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  metadata-ageThe age in seconds of the current producer metadata being used.kafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  connection-close-rateConnections closed per second in the window.kafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  connection-creation-rateNew connections established per second in the window.kafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  network-io-rateThe average number of network operations (reads or writes) on all connections per second.kafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  outgoing-byte-rateThe average number of outgoing bytes sent per second to all servers.kafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  request-rateThe average number of requests sent per second.kafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  request-size-avgThe average size of all requests in the window.kafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  request-size-maxThe maximum size of any request sent in the window.kafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  incoming-byte-rateBytes/second read off all socketskafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  response-rateResponses received sent per second.kafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  select-rateNumber of times the I/O layer checked for new I/O to perform per secondkafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  io-wait-time-ns-avgThe average length of time the I/O thread spent waiting for a socket ready for reads or writes in nanoseconds.kafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  io-wait-ratioThe fraction of time the I/O thread spent waiting.kafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  io-time-ns-avgThe average length of time for I/O per select call in nanoseconds.kafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  io-ratioThe fraction of time the I/O thread spent doing I/Okafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  connection-countThe current number of active connections.kafka.producer:type=producer-metrics,client-id=([-.\w]+)
                  outgoing-byte-rateThe average number of outgoing bytes sent per second for a node.kafka.producer:type=producer-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)
                  request-rateThe average number of requests sent per second for a node.kafka.producer:type=producer-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)
                  request-size-avgThe average size of all requests in the window for a node.kafka.producer:type=producer-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)
                  request-size-maxThe maximum size of any request sent in the window for a node.kafka.producer:type=producer-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)
                  incoming-byte-rateThe average number of responses received per second for a node.kafka.producer:type=producer-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)
                  request-latency-avgThe average request latency in ms for a node.kafka.producer:type=producer-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)
                  request-latency-maxThe maximum request latency in ms for a node.kafka.producer:type=producer-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)
                  response-rateResponses received sent per second for a node.kafka.producer:type=producer-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)
                  record-send-rateThe average number of records sent per second for a topic.kafka.producer:type=producer-topic-metrics,client-id=([-.\w]+),topic=([-.\w]+)
                  byte-rateThe average number of bytes sent per second for a topic.kafka.producer:type=producer-topic-metrics,client-id=([-.\w]+),topic=([-.\w]+)
                  compression-rateThe average compression rate of record batches for a topic.kafka.producer:type=producer-topic-metrics,client-id=([-.\w]+),topic=([-.\w]+)
                  record-retry-rateThe average per-second number of retried record sends for a topickafka.producer:type=producer-topic-metrics,client-id=([-.\w]+),topic=([-.\w]+)
                  record-error-rateThe average per-second number of record sends that resulted in errors for a topic.kafka.producer:type=producer-topic-metrics,client-id=([-.\w]+),topic=([-.\w]+)
                  + +We recommend monitor GC time and other stats and various server stats such as CPU utilization, I/O service time, etc. + +On the client side, we recommend monitor the message/byte rate (global and per topic), request rate/size/time, and on the consumer side, max lag in messages among all partitions and min fetch request rate. For a consumer to keep up, max lag needs to be less than a threshold and min fetch rate needs to be larger than 0. + +

                  Audit

                  +The final alerting we do is on the correctness of the data delivery. We audit that every message that is sent is consumed by all consumers and measure the lag for this to occur. For important topics we alert if a certain completeness is not achieved in a certain time period. The details of this are discussed in KAFKA-260. + +

                  6.7 ZooKeeper

                  + +

                  Stable version

                  +At LinkedIn, we are running ZooKeeper 3.3.*. Version 3.3.3 has known serious issues regarding ephemeral node deletion and session expirations. After running into those issues in production, we upgraded to 3.3.4 and have been running that smoothly for over a year now. + +

                  Operationalizing ZooKeeper

                  +Operationally, we do the following for a healthy ZooKeeper installation: +
                    +
                  • Redundancy in the physical/hardware/network layout: try not to put them all in the same rack, decent (but don't go nuts) hardware, try to keep redundant power and network paths, etc.
                  • +
                  • I/O segregation: if you do a lot of write type traffic you'll almost definitely want the transaction logs on a different disk group than application logs and snapshots (the write to the ZooKeeper service has a synchronous write to disk, which can be slow).
                  • +
                  • Application segregation: Unless you really understand the application patterns of other apps that you want to install on the same box, it can be a good idea to run ZooKeeper in isolation (though this can be a balancing act with the capabilities of the hardware).
                  • +
                  • Use care with virtualization: It can work, depending on your cluster layout and read/write patterns and SLAs, but the tiny overheads introduced by the virtualization layer can add up and throw off ZooKeeper, as it can be very time sensitive
                  • +
                  • ZooKeeper configuration and monitoring: It's java, make sure you give it 'enough' heap space (We usually run them with 3-5G, but that's mostly due to the data set size we have here). Unfortunately we don't have a good formula for it. As far as monitoring, both JMX and the 4 letter words (4lw) commands are very useful, they do overlap in some cases (and in those cases we prefer the 4 letter commands, they seem more predictable, or at the very least, they work better with the LI monitoring infrastructure)
                  • +
                  • Don't overbuild the cluster: large clusters, especially in a write heavy usage pattern, means a lot of intracluster communication (quorums on the writes and subsequent cluster member updates), but don't underbuild it (and risk swamping the cluster).
                  • +
                  • Try to run on a 3-5 node cluster: ZooKeeper writes use quorums and inherently that means having an odd number of machines in a cluster. Remember that a 5 node cluster will cause writes to slow down compared to a 3 node cluster, but will allow more fault tolerance.
                  • +
                  +Overall, we try to keep the ZooKeeper system as small as will handle the load (plus standard growth capacity planning) and as simple as possible. We try not to do anything fancy with the configuration or application layout as compared to the official release as well as keep it as self contained as possible. For these reasons, we tend to skip the OS packaged versions, since it has a tendency to try to put things in the OS standard hierarchy, which can be 'messy', for want of a better way to word it. diff --git a/docs/quickstart.html b/docs/quickstart.html new file mode 100644 index 0000000000000..18f610bd0c38a --- /dev/null +++ b/docs/quickstart.html @@ -0,0 +1,189 @@ + + +

                  1.3 Quick Start

                  + +This tutorial assumes you are starting fresh and have no existing Kafka or ZooKeeper data. + +

                  Step 1: Download the code

                  + +Download the 0.8.2.0 release and un-tar it. + +
                  +> tar -xzf kafka_2.10-0.8.2.0.tgz
                  +> cd kafka_2.10-0.8.2.0
                  +
                  + +

                  Step 2: Start the server

                  + +

                  +Kafka uses ZooKeeper so you need to first start a ZooKeeper server if you don't already have one. You can use the convenience script packaged with kafka to get a quick-and-dirty single-node ZooKeeper instance. + +

                  +> bin/zookeeper-server-start.sh config/zookeeper.properties
                  +[2013-04-22 15:01:37,495] INFO Reading configuration from: config/zookeeper.properties (org.apache.zookeeper.server.quorum.QuorumPeerConfig)
                  +...
                  +
                  + +Now start the Kafka server: +
                  +> bin/kafka-server-start.sh config/server.properties
                  +[2013-04-22 15:01:47,028] INFO Verifying properties (kafka.utils.VerifiableProperties)
                  +[2013-04-22 15:01:47,051] INFO Property socket.send.buffer.bytes is overridden to 1048576 (kafka.utils.VerifiableProperties)
                  +...
                  +
                  + +

                  Step 3: Create a topic

                  + +Let's create a topic named "test" with a single partition and only one replica: +
                  +> bin/kafka-topics.sh --create --zookeeper localhost:2181 --replication-factor 1 --partitions 1 --topic test
                  +
                  + +We can now see that topic if we run the list topic command: +
                  +> bin/kafka-topics.sh --list --zookeeper localhost:2181
                  +test
                  +
                  +Alternatively, instead of manually creating topics you can also configure your brokers to auto-create topics when a non-existent topic is published to. + +

                  Step 4: Send some messages

                  + +Kafka comes with a command line client that will take input from a file or from standard input and send it out as messages to the Kafka cluster. By default each line will be sent as a separate message. +

                  +Run the producer and then type a few messages into the console to send to the server. + +

                  +> bin/kafka-console-producer.sh --broker-list localhost:9092 --topic test 
                  +This is a message
                  +This is another message
                  +
                  + +

                  Step 5: Start a consumer

                  + +Kafka also has a command line consumer that will dump out messages to standard output. + +
                  +> bin/kafka-console-consumer.sh --zookeeper localhost:2181 --topic test --from-beginning
                  +This is a message
                  +This is another message
                  +
                  +

                  +If you have each of the above commands running in a different terminal then you should now be able to type messages into the producer terminal and see them appear in the consumer terminal. +

                  +

                  +All of the command line tools have additional options; running the command with no arguments will display usage information documenting them in more detail. +

                  + +

                  Step 6: Setting up a multi-broker cluster

                  + +So far we have been running against a single broker, but that's no fun. For Kafka, a single broker is just a cluster of size one, so nothing much changes other than starting a few more broker instances. But just to get feel for it, let's expand our cluster to three nodes (still all on our local machine). +

                  +First we make a config file for each of the brokers: +

                  +> cp config/server.properties config/server-1.properties 
                  +> cp config/server.properties config/server-2.properties
                  +
                  + +Now edit these new files and set the following properties: +
                  + 
                  +config/server-1.properties:
                  +    broker.id=1
                  +    port=9093
                  +    log.dir=/tmp/kafka-logs-1
                  + 
                  +config/server-2.properties:
                  +    broker.id=2
                  +    port=9094
                  +    log.dir=/tmp/kafka-logs-2
                  +
                  +The broker.id property is the unique and permanent name of each node in the cluster. We have to override the port and log directory only because we are running these all on the same machine and we want to keep the brokers from all trying to register on the same port or overwrite each others data. +

                  +We already have Zookeeper and our single node started, so we just need to start the two new nodes: +

                  +> bin/kafka-server-start.sh config/server-1.properties &
                  +...
                  +> bin/kafka-server-start.sh config/server-2.properties &
                  +...
                  +
                  + +Now create a new topic with a replication factor of three: +
                  +> bin/kafka-topics.sh --create --zookeeper localhost:2181 --replication-factor 3 --partitions 1 --topic my-replicated-topic
                  +
                  + +Okay but now that we have a cluster how can we know which broker is doing what? To see that run the "describe topics" command: +
                  +> bin/kafka-topics.sh --describe --zookeeper localhost:2181 --topic my-replicated-topic
                  +Topic:my-replicated-topic	PartitionCount:1	ReplicationFactor:3	Configs:
                  +	Topic: my-replicated-topic	Partition: 0	Leader: 1	Replicas: 1,2,0	Isr: 1,2,0
                  +
                  +Here is an explanation of output. The first line gives a summary of all the partitions, each additional line gives information about one partition. Since we have only one partition for this topic there is only one line. +
                    +
                  • "leader" is the node responsible for all reads and writes for the given partition. Each node will be the leader for a randomly selected portion of the partitions. +
                  • "replicas" is the list of nodes that replicate the log for this partition regardless of whether they are the leader or even if they are currently alive. +
                  • "isr" is the set of "in-sync" replicas. This is the subset of the replicas list that is currently alive and caught-up to the leader. +
                  +Note that in my example node 1 is the leader for the only partition of the topic. +

                  +We can run the same command on the original topic we created to see where it is: +

                  +> bin/kafka-topics.sh --describe --zookeeper localhost:2181 --topic test
                  +Topic:test	PartitionCount:1	ReplicationFactor:1	Configs:
                  +	Topic: test	Partition: 0	Leader: 0	Replicas: 0	Isr: 0
                  +
                  +So there is no surprise there—the original topic has no replicas and is on server 0, the only server in our cluster when we created it. +

                  +Let's publish a few messages to our new topic: +

                  +> bin/kafka-console-producer.sh --broker-list localhost:9092 --topic my-replicated-topic
                  +...
                  +my test message 1
                  +my test message 2
                  +^C 
                  +
                  +Now let's consume these messages: +
                  +> bin/kafka-console-consumer.sh --zookeeper localhost:2181 --from-beginning --topic my-replicated-topic
                  +...
                  +my test message 1
                  +my test message 2
                  +^C
                  +
                  + +Now let's test out fault-tolerance. Broker 1 was acting as the leader so let's kill it: +
                  +> ps | grep server-1.properties
                  +7564 ttys002    0:15.91 /System/Library/Frameworks/JavaVM.framework/Versions/1.6/Home/bin/java...
                  +> kill -9 7564
                  +
                  + +Leadership has switched to one of the slaves and node 1 is no longer in the in-sync replica set: +
                  +> bin/kafka-topics.sh --describe --zookeeper localhost:2181 --topic my-replicated-topic
                  +Topic:my-replicated-topic	PartitionCount:1	ReplicationFactor:3	Configs:
                  +	Topic: my-replicated-topic	Partition: 0	Leader: 2	Replicas: 1,2,0	Isr: 2,0
                  +
                  +But the messages are still be available for consumption even though the leader that took the writes originally is down: +
                  +> bin/kafka-console-consumer.sh --zookeeper localhost:2181 --from-beginning --topic my-replicated-topic
                  +...
                  +my test message 1
                  +my test message 2
                  +^C
                  +
                  diff --git a/docs/upgrade.html b/docs/upgrade.html new file mode 100644 index 0000000000000..4b7033a34fbfd --- /dev/null +++ b/docs/upgrade.html @@ -0,0 +1,44 @@ + + +

                  1.5 Upgrading From Previous Versions

                  + +

                  Upgrading from 0.8.0, 0.8.1.X or 0.8.2.X to 0.8.3.0

                  + +0.8.3.0 has an inter-broker protocol change from previous versions. For a rolling upgrade: +
                    +
                  1. Update server.properties file on all brokers and add the following property: inter.broker.protocol.version=0.8.2.X
                  2. +
                  3. Upgrade the brokers. This can be done a broker at a time by simply bringing it down, updating the code, and restarting it.
                  4. +
                  5. Once the entire cluster is upgraded, bump the protocol version by editing inter.broker.protocol.version and setting it to 0.8.3.0.
                  6. +
                  7. Restart the brokers one by one for the new protocol version to take effect
                  8. +
                  + +Note: If you are willing to accept downtime, you can simply take all the brokers down, update the code and start all of them. They will start with the new protocol by default. + +Note: Bumping the protocol version and restarting can be done any time after the brokers were upgraded. It does not have to be immediately after. + +

                  Upgrading from 0.8.1 to 0.8.2.0

                  + +0.8.2.0 is fully compatible with 0.8.1. The upgrade can be done one broker at a time by simply bringing it down, updating the code, and restarting it. + +

                  Upgrading from 0.8.0 to 0.8.1

                  + +0.8.1 is fully compatible with 0.8. The upgrade can be done one broker at a time by simply bringing it down, updating the code, and restarting it. + +

                  Upgrading from 0.7

                  + +0.8, the release in which added replication, was our first backwards-incompatible release: major changes were made to the API, ZooKeeper data structures, and protocol, and configuration. The upgrade from 0.7 to 0.8.x requires a special tool for migration. This migration can be done without downtime. diff --git a/docs/uses.html b/docs/uses.html new file mode 100644 index 0000000000000..aa87d07de1239 --- /dev/null +++ b/docs/uses.html @@ -0,0 +1,56 @@ + + +

                  1.2 Use Cases

                  + +Here is a description of a few of the popular use cases for Apache Kafka. For an overview of a number of these areas in action, see this blog post. + +

                  Messaging

                  + +Kafka works well as a replacement for a more traditional message broker. Message brokers are used for a variety of reasons (to decouple processing from data producers, to buffer unprocessed messages, etc). In comparison to most messaging systems Kafka has better throughput, built-in partitioning, replication, and fault-tolerance which makes it a good solution for large scale message processing applications. +

                  +In our experience messaging uses are often comparatively low-throughput, but may require low end-to-end latency and often depend on the strong durability guarantees Kafka provides. +

                  +In this domain Kafka is comparable to traditional messaging systems such as ActiveMQ or RabbitMQ. + +

                  Website Activity Tracking

                  + +The original use case for Kafka was to be able to rebuild a user activity tracking pipeline as a set of real-time publish-subscribe feeds. This means site activity (page views, searches, or other actions users may take) is published to central topics with one topic per activity type. These feeds are available for subscription for a range of use cases including real-time processing, real-time monitoring, and loading into Hadoop or offline data warehousing systems for offline processing and reporting. +

                  +Activity tracking is often very high volume as many activity messages are generated for each user page view. + +

                  Metrics

                  + +Kafka is often used for operational monitoring data. This involves aggregating statistics from distributed applications to produce centralized feeds of operational data. + +

                  Log Aggregation

                  + +Many people use Kafka as a replacement for a log aggregation solution. Log aggregation typically collects physical log files off servers and puts them in a central place (a file server or HDFS perhaps) for processing. Kafka abstracts away the details of files and gives a cleaner abstraction of log or event data as a stream of messages. This allows for lower-latency processing and easier support for multiple data sources and distributed data consumption. + +In comparison to log-centric systems like Scribe or Flume, Kafka offers equally good performance, stronger durability guarantees due to replication, and much lower end-to-end latency. + +

                  Stream Processing

                  + +Many users end up doing stage-wise processing of data where data is consumed from topics of raw data and then aggregated, enriched, or otherwise transformed into new Kafka topics for further consumption. For example a processing flow for article recommendation might crawl article content from RSS feeds and publish it to an "articles" topic; further processing might help normalize or deduplicate this content to a topic of cleaned article content; a final stage might attempt to match this content to users. This creates a graph of real-time data flow out of the individual topics. Storm and Samza are popular frameworks for implementing these kinds of transformations. + +

                  Event Sourcing

                  + +Event sourcing is a style of application design where state changes are logged as a time-ordered sequence of records. Kafka's support for very large stored log data makes it an excellent backend for an application built in this style. + +

                  Commit Log

                  + +Kafka can serve as a kind of external commit-log for a distributed system. The log helps replicate data between nodes and acts as a re-syncing mechanism for failed nodes to restore their data. The log compaction feature in Kafka helps support this usage. In this usage Kafka is similar to Apache BookKeeper project. 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/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 8f6f2bf7bbd27..faeaf885efed0 100644 --- a/gradle.properties +++ b/gradle.properties @@ -14,10 +14,7 @@ # limitations under the License. group=org.apache.kafka -version=0.8.2-SNAPSHOT -scalaVersion=2.10.1 +version=0.9.0.0-SNAPSHOT +scalaVersion=2.10.5 task=build - -mavenUrl= -mavenUsername= -mavenPassword= +org.gradle.jvmargs=-XX:MaxPermSize=512m -Xmx1024m -Xss2m diff --git a/gradle/buildscript.gradle b/gradle/buildscript.gradle index 5e45c06e8bb8b..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.10.0' -} 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/kafka-merge-pr.py b/kafka-merge-pr.py new file mode 100644 index 0000000000000..078708e0590a1 --- /dev/null +++ b/kafka-merge-pr.py @@ -0,0 +1,474 @@ +#!/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" + +DEFAULT_FIX_VERSION = os.environ.get("DEFAULT_FIX_VERSION", "0.9.0.0") + +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(): + if original_head != get_current_branch(): + 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) + +def get_current_branch(): + return run_cmd("git rev-parse --abbrev-ref HEAD").replace("\n", "") + +# 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 = raw_input( + "Enter primary author in the format of \"name \" [%s]: " % + distinct_authors[0]) + if primary_author == "": + primary_author = distinct_authors[0] + + reviewers = raw_input( + "Enter reviewers in the format of \"name1 , name2 \": ").strip() + + commits = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, + '--pretty=format:%h [%an] %s']).split("\n") + + if len(commits) > 1: + result = raw_input("List pull request commits in squashed commit message? (y/n): ") + if result.lower() == "y": + should_list_commits = True + else: + should_list_commits = False + else: + should_list_commits = False + + 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 (reviewers != ""): + merge_message_flags += ["-m", "Reviewers: %s" % reviewers] + + 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 + close_line = "Closes #%s from %s" % (pr_num, pr_repo_desc) + if should_list_commits: + close_line += " and squashes the following commits:" + merge_message_flags += ["-m", close_line] + + if should_list_commits: + merge_message_flags += ["-m", "\n".join(commits)] + + 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: + versions = filter(lambda x: x == DEFAULT_FIX_VERSION, versions) + if len(versions) > 0: + return versions[0] + else: + return None + else: + versions = filter(lambda x: x.startswith(branch), versions) + if len(versions) > 0: + return versions[-1] + else: + return None + + +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) + + version_names = map(lambda x: x.name, versions) + default_fix_versions = map(lambda x: fix_version_from_branch(x, version_names), merge_branches) + default_fix_versions = filter(lambda x: x != None, 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] + resolution = filter(lambda r: r.raw['name'] == "Fixed", asf_jira.resolutions())[0] + asf_jira.transition_issue( + jira_id, resolve["id"], fixVersions = jira_fix_versions, + comment = comment, resolution = {'id': resolution.raw['id']}) + + 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 = get_current_branch() + + 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"] + + pr_title = pr["title"] + commit_title = raw_input("Commit title [%s]: " % pr_title.encode("utf-8")).decode("utf-8") + if commit_title == "": + commit_title = pr_title + + # Decide whether to use the modified title or not + modified_title = standardize_jira_ref(commit_title) + if modified_title != commit_title: + print "I've re-written the title as follows to match the standard format:" + print "Original: %s" % commit_title + print "Modified: %s" % modified_title + result = raw_input("Would you like to use the modified title? (y/n): ") + if result.lower() == "y": + commit_title = modified_title + print "Using modified title:" + else: + print "Using original title:" + print commit_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 ("PR title\t%s\nCommit title\t%s\nSource\t\t%s\nTarget\t\t%s\nURL\t\t%s" % ( + pr_title, commit_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, commit_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(commit_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' to install." + print "Exiting without trying to close the associated JIRA." + +if __name__ == "__main__": + import doctest + (failure_count, test_count) = doctest.testmod() + if (failure_count): + exit(-1) + + main() diff --git a/kafka-patch-review.py b/kafka-patch-review.py index b7f132f9d210b..94873c3c29cf2 100644 --- a/kafka-patch-review.py +++ b/kafka-patch-review.py @@ -1,4 +1,21 @@ #!/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 @@ -7,22 +24,31 @@ import datetime import tempfile import commands +import getpass from jira.client import 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')) - return jira_config + 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(): +def get_jira(jira_config): options = { 'server': 'https://issues.apache.org/jira' } - - jira_config = get_jira_config() 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): @@ -81,6 +107,46 @@ def main(): p=os.popen(git_remote_update) p.close() + # 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" @@ -116,14 +182,7 @@ def main(): if opt.debug: print 'rb url=',rb_url - git_command="git format-patch " + opt.branch + " --stdout > " + patch_file - if opt.debug: - print git_command - p=os.popen(git_command) - p.close() - 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) @@ -146,8 +205,6 @@ def main(): for t in transitions: transitionsMap[t['name']] = t['id'] - jira_config = get_jira_config() - if('Submit Patch' in transitionsMap): jira.transition_issue(issue, transitionsMap['Submit Patch'] , assignee={'name': jira_config['user']} ) 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..2baef06e1a8e3 --- /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, Integer.toString(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/scala.gradle b/scala.gradle index 6adf9af7dbbe7..5eb2a656a05f5 100644 --- a/scala.gradle +++ b/scala.gradle @@ -1,7 +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. + if (!hasProperty('scalaVersion')) { - ext.scalaVersion = '2.10.1' + ext.scalaVersion = '2.10.5' } -ext.defaultScalaVersion = '2.10.1' +ext.defaultScalaVersion = '2.10.5' if (scalaVersion.startsWith('2.10')) { ext.baseScalaVersion = '2.10' } else if (scalaVersion.startsWith('2.11')) { diff --git a/settings.gradle b/settings.gradle index 83f764e6a4a15..357305ba23e8c 100644 --- a/settings.gradle +++ b/settings.gradle @@ -14,4 +14,5 @@ // limitations under the License. apply from: file('scala.gradle') -include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients' +include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients', 'tools', 'streams', 'log4j-appender', + 'copycat:api', 'copycat:runtime', 'copycat:json', 'copycat:file' \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java new file mode 100644 index 0000000000000..d274fb97d0c59 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java @@ -0,0 +1,156 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

                  + * http://www.apache.org/licenses/LICENSE-2.0 + *

                  + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams; + +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.utils.SystemTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.processor.TopologyBuilder; +import org.apache.kafka.streams.processor.internals.StreamThread; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Kafka Streaming allows for performing continuous computation on input coming from one or more input topics and + * sends output to zero or more output topics. + *

                  + * This processing is defined by using the {@link TopologyBuilder} class or its superclass KStreamBuilder to specify + * the transformation. + * The {@link KafkaStreaming} instance will be responsible for the lifecycle of these processors. It will instantiate and + * start one or more of these processors to process the Kafka partitions assigned to this particular instance. + *

                  + * This streaming instance will co-ordinate with any other instances (whether in this same process, on other processes + * on this machine, or on remote machines). These processes will divide up the work so that all partitions are being + * consumed. If instances are added or die, the corresponding {@link StreamThread} instances will be shutdown or + * started in the appropriate processes to balance processing load. + *

                  + * Internally the {@link KafkaStreaming} instance contains a normal {@link org.apache.kafka.clients.producer.KafkaProducer KafkaProducer} + * and {@link org.apache.kafka.clients.consumer.KafkaConsumer KafkaConsumer} instance that is used for reading input and writing output. + *

                  + * A simple example might look like this: + *

                  + *    Map<String, Object> props = new HashMap<>();
                  + *    props.put("bootstrap.servers", "localhost:4242");
                  + *    props.put("key.deserializer", StringDeserializer.class);
                  + *    props.put("value.deserializer", StringDeserializer.class);
                  + *    props.put("key.serializer", StringSerializer.class);
                  + *    props.put("value.serializer", IntegerSerializer.class);
                  + *    props.put("timestamp.extractor", MyTimestampExtractor.class);
                  + *    StreamingConfig config = new StreamingConfig(props);
                  + *
                  + *    KStreamBuilder builder = new KStreamBuilder();
                  + *    builder.from("topic1").mapValue(value -> value.length()).to("topic2");
                  + *
                  + *    KafkaStreaming streaming = new KafkaStreaming(builder, config);
                  + *    streaming.start();
                  + * 
                  + * + */ +public class KafkaStreaming { + + private static final Logger log = LoggerFactory.getLogger(KafkaStreaming.class); + private static final AtomicInteger STREAMING_CLIENT_ID_SEQUENCE = new AtomicInteger(1); + private static final String JMX_PREFIX = "kafka.streaming"; + + private final Time time; + + // container states + private static final int CREATED = 0; + private static final int RUNNING = 1; + private static final int STOPPED = 2; + private int state = CREATED; + + private final StreamThread[] threads; + + private String clientId; + private final Metrics metrics; + + public KafkaStreaming(TopologyBuilder builder, StreamingConfig config) throws Exception { + // create the metrics + this.time = new SystemTime(); + + MetricConfig metricConfig = new MetricConfig().samples(config.getInt(StreamingConfig.METRICS_NUM_SAMPLES_CONFIG)) + .timeWindow(config.getLong(StreamingConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG), + TimeUnit.MILLISECONDS); + clientId = config.getString(StreamingConfig.CLIENT_ID_CONFIG); + if (clientId.length() <= 0) + clientId = "streaming-" + STREAMING_CLIENT_ID_SEQUENCE.getAndIncrement(); + List reporters = config.getConfiguredInstances(StreamingConfig.METRIC_REPORTER_CLASSES_CONFIG, + MetricsReporter.class); + reporters.add(new JmxReporter(JMX_PREFIX)); + this.metrics = new Metrics(metricConfig, reporters, time); + + this.threads = new StreamThread[config.getInt(StreamingConfig.NUM_STREAM_THREADS_CONFIG)]; + for (int i = 0; i < this.threads.length; i++) { + this.threads[i] = new StreamThread(builder, config, this.clientId, this.metrics, this.time); + } + } + + /** + * Start the stream process by starting all its threads + */ + public synchronized void start() { + log.debug("Starting Kafka Stream process"); + + if (state == CREATED) { + for (StreamThread thread : threads) + thread.start(); + + state = RUNNING; + + log.info("Started Kafka Stream process"); + } else { + throw new IllegalStateException("This process was already started."); + } + } + + /** + * Shutdown this stream process by signaling the threads to stop, + * wait for them to join and clean up the process instance. + */ + public synchronized void close() { + log.debug("Stopping Kafka Stream process"); + + if (state == RUNNING) { + // signal the threads to stop and wait + for (StreamThread thread : threads) + thread.close(); + + for (StreamThread thread : threads) { + try { + thread.join(); + } catch (InterruptedException ex) { + Thread.interrupted(); + } + } + + state = STOPPED; + + log.info("Stopped Kafka Stream process"); + } else { + throw new IllegalStateException("This process has not started yet."); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamingConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamingConfig.java new file mode 100644 index 0000000000000..93df4c2494507 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/StreamingConfig.java @@ -0,0 +1,229 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

                  + * http://www.apache.org/licenses/LICENSE-2.0 + *

                  + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams; + +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; +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 java.util.Map; + +import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; + +public class StreamingConfig extends AbstractConfig { + + private static final ConfigDef CONFIG; + + /** state.dir */ + public static final String STATE_DIR_CONFIG = "state.dir"; + private static final String STATE_DIR_DOC = "Directory location for state store."; + + /** commit.interval.ms */ + public static final String COMMIT_INTERVAL_MS_CONFIG = "commit.interval.ms"; + private static final String COMMIT_INTERVAL_MS_DOC = "The frequency with which to save the position of the processor."; + + /** poll.ms */ + public static final String POLL_MS_CONFIG = "poll.ms"; + private static final String POLL_MS_DOC = "The amount of time in milliseconds to block waiting for input."; + + /** num.stream.threads */ + public static final String NUM_STREAM_THREADS_CONFIG = "num.stream.threads"; + private static final String NUM_STREAM_THREADS_DOC = "The number of threads to execute stream processing."; + + /** buffered.records.per.partition */ + public static final String BUFFERED_RECORDS_PER_PARTITION_CONFIG = "buffered.records.per.partition"; + private static final String BUFFERED_RECORDS_PER_PARTITION_DOC = "The maximum number of records to buffer per partition."; + + /** state.cleanup.delay */ + public static final String STATE_CLEANUP_DELAY_MS_CONFIG = "state.cleanup.delay.ms"; + private static final String STATE_CLEANUP_DELAY_MS_DOC = "The amount of time in milliseconds to wait before deleting state when a partition has migrated."; + + /** total.records.to.process */ + public static final String TOTAL_RECORDS_TO_PROCESS = "total.records.to.process"; + private static final String TOTAL_RECORDS_TO_DOC = "Exit after processing this many records."; + + /** window.time.ms */ + public static final String WINDOW_TIME_MS_CONFIG = "window.time.ms"; + private static final String WINDOW_TIME_MS_DOC = "Setting this to a non-negative value will cause the processor to get called " + + "with this frequency even if there is no message."; + + /** timestamp.extractor */ + public static final String TIMESTAMP_EXTRACTOR_CLASS_CONFIG = "timestamp.extractor"; + private static final String TIMESTAMP_EXTRACTOR_CLASS_DOC = "Timestamp extractor class that implements the TimestampExtractor interface."; + + /** client.id */ + public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG; + + /** key.serializer */ + public static final String KEY_SERIALIZER_CLASS_CONFIG = ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG; + + /** value.serializer */ + public static final String VALUE_SERIALIZER_CLASS_CONFIG = ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG; + + /** key.deserializer */ + public static final String KEY_DESERIALIZER_CLASS_CONFIG = ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; + + /** value.deserializer */ + public static final String VALUE_DESERIALIZER_CLASS_CONFIG = ConsumerConfig.VALUE_DESERIALIZER_CLASS_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; + + /** + * bootstrap.servers + */ + public static final String BOOTSTRAP_SERVERS_CONFIG = CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; + + private static final String SYSTEM_TEMP_DIRECTORY = System.getProperty("java.io.tmpdir"); + + static { + CONFIG = new ConfigDef().define(CLIENT_ID_CONFIG, + Type.STRING, + "", + Importance.MEDIUM, + CommonClientConfigs.CLIENT_ID_DOC) + .define(STATE_DIR_CONFIG, + Type.STRING, + SYSTEM_TEMP_DIRECTORY, + Importance.MEDIUM, + STATE_DIR_DOC) + .define(COMMIT_INTERVAL_MS_CONFIG, + Type.LONG, + 30000, + Importance.HIGH, + COMMIT_INTERVAL_MS_DOC) + .define(POLL_MS_CONFIG, + Type.LONG, + 100, + Importance.LOW, + POLL_MS_DOC) + .define(NUM_STREAM_THREADS_CONFIG, + Type.INT, + 1, + Importance.LOW, + NUM_STREAM_THREADS_DOC) + .define(BUFFERED_RECORDS_PER_PARTITION_CONFIG, + Type.INT, + 1000, + Importance.LOW, + BUFFERED_RECORDS_PER_PARTITION_DOC) + .define(STATE_CLEANUP_DELAY_MS_CONFIG, + Type.LONG, + 60000, + Importance.LOW, + STATE_CLEANUP_DELAY_MS_DOC) + .define(TOTAL_RECORDS_TO_PROCESS, + Type.LONG, + -1L, + Importance.LOW, + TOTAL_RECORDS_TO_DOC) + .define(WINDOW_TIME_MS_CONFIG, + Type.LONG, + -1L, + Importance.MEDIUM, + WINDOW_TIME_MS_DOC) + .define(KEY_SERIALIZER_CLASS_CONFIG, + Type.CLASS, + Importance.HIGH, + ProducerConfig.KEY_SERIALIZER_CLASS_DOC) + .define(VALUE_SERIALIZER_CLASS_CONFIG, + Type.CLASS, + Importance.HIGH, + ProducerConfig.VALUE_SERIALIZER_CLASS_DOC) + .define(KEY_DESERIALIZER_CLASS_CONFIG, + Type.CLASS, + Importance.HIGH, + ConsumerConfig.KEY_DESERIALIZER_CLASS_DOC) + .define(VALUE_DESERIALIZER_CLASS_CONFIG, + Type.CLASS, + Importance.HIGH, + ConsumerConfig.VALUE_DESERIALIZER_CLASS_DOC) + .define(TIMESTAMP_EXTRACTOR_CLASS_CONFIG, + Type.CLASS, + Importance.HIGH, + TIMESTAMP_EXTRACTOR_CLASS_DOC) + .define(BOOTSTRAP_SERVERS_CONFIG, + Type.STRING, + Importance.HIGH, + CommonClientConfigs.BOOSTRAP_SERVERS_DOC) + .define(METRIC_REPORTER_CLASSES_CONFIG, + Type.LIST, + "", + Importance.LOW, + CommonClientConfigs.METRIC_REPORTER_CLASSES_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); + } + + public StreamingConfig(Map props) { + super(CONFIG, props); + } + + public Map getConsumerConfigs() { + Map props = this.originals(); + + // set consumer default property values + props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + props.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "range"); + + // remove properties that are not required for consumers + props.remove(StreamingConfig.KEY_SERIALIZER_CLASS_CONFIG); + props.remove(StreamingConfig.VALUE_SERIALIZER_CLASS_CONFIG); + props.remove(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG); + + return props; + } + + public Map getProducerConfigs() { + Map props = this.originals(); + + // set producer default property values + props.put(ProducerConfig.LINGER_MS_CONFIG, "100"); + + // remove properties that are not required for producers + props.remove(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG); + props.remove(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG); + props.remove(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG); + + return props; + } + + public static void main(String[] args) { + System.out.println(CONFIG.toHtmlTable()); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamingMetrics.java b/streams/src/main/java/org/apache/kafka/streams/StreamingMetrics.java new file mode 100644 index 0000000000000..ebf80b3f9e3e8 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/StreamingMetrics.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.streams; + +import org.apache.kafka.common.metrics.Sensor; + +public interface StreamingMetrics { + + Sensor addLatencySensor(String scopeName, String entityName, String operationName, String... tags); + + void recordLatency(Sensor sensor, long startNs, long endNs); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/examples/KStreamJob.java b/streams/src/main/java/org/apache/kafka/streams/examples/KStreamJob.java new file mode 100644 index 0000000000000..87368c1c46520 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/examples/KStreamJob.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.streams.examples; + +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.StreamingConfig; +import org.apache.kafka.streams.KafkaStreaming; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KeyValue; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.Predicate; + +import java.util.Properties; + +public class KStreamJob { + + public static void main(String[] args) throws Exception { + Properties props = new Properties(); + props.put(StreamingConfig.CLIENT_ID_CONFIG, "Example-KStream-Job"); + props.put(StreamingConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); + props.put(StreamingConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); + props.put(StreamingConfig.VALUE_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class); + props.put(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, WallclockTimestampExtractor.class); + StreamingConfig config = new StreamingConfig(props); + + KStreamBuilder builder = new KStreamBuilder(); + + KStream stream1 = builder.from("topic1"); + + KStream stream2 = + stream1.map(new KeyValueMapper>() { + @Override + public KeyValue apply(String key, String value) { + return new KeyValue<>(key, new Integer(value)); + } + }).filter(new Predicate() { + @Override + public boolean test(String key, Integer value) { + return true; + } + }); + + KStream[] streams = stream2.branch( + new Predicate() { + @Override + public boolean test(String key, Integer value) { + return (value % 2) == 0; + } + }, + new Predicate() { + @Override + public boolean test(String key, Integer value) { + return true; + } + } + ); + + streams[0].to("topic2"); + streams[1].to("topic3"); + + KafkaStreaming kstream = new KafkaStreaming(builder, config); + kstream.start(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java b/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java new file mode 100644 index 0000000000000..92e6284e73cfe --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.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.streams.examples; + +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streams.KafkaStreaming; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.TopologyBuilder; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.StreamingConfig; +import org.apache.kafka.streams.state.Entry; +import org.apache.kafka.streams.state.InMemoryKeyValueStore; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.KeyValueStore; + +import java.util.Properties; + +public class ProcessorJob { + + private static class MyProcessorSupplier implements ProcessorSupplier { + + @Override + public Processor get() { + return new Processor() { + private ProcessorContext context; + private KeyValueStore kvStore; + + @Override + public void init(ProcessorContext context) { + this.context = context; + this.context.schedule(1000); + this.kvStore = new InMemoryKeyValueStore<>("local-state", context); + } + + @Override + public void process(String key, String value) { + Integer oldValue = this.kvStore.get(key); + Integer newValue = Integer.parseInt(value); + if (oldValue == null) { + this.kvStore.put(key, newValue); + } else { + this.kvStore.put(key, oldValue + newValue); + } + + context.commit(); + } + + @Override + public void punctuate(long timestamp) { + KeyValueIterator iter = this.kvStore.all(); + + while (iter.hasNext()) { + Entry entry = iter.next(); + + System.out.println("[" + entry.key() + ", " + entry.value() + "]"); + + context.forward(entry.key(), entry.value()); + } + + iter.close(); + } + + @Override + public void close() { + this.kvStore.close(); + } + }; + } + } + + public static void main(String[] args) throws Exception { + Properties props = new Properties(); + props.put(StreamingConfig.CLIENT_ID_CONFIG, "Example-Processor-Job"); + props.put(StreamingConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); + props.put(StreamingConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); + props.put(StreamingConfig.VALUE_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class); + props.put(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG, IntegerDeserializer.class); + props.put(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, WallclockTimestampExtractor.class); + StreamingConfig config = new StreamingConfig(props); + + TopologyBuilder builder = new TopologyBuilder(); + + builder.addSource("SOURCE", new StringDeserializer(), new StringDeserializer(), "topic-source"); + + builder.addProcessor("PROCESS", new MyProcessorSupplier(), "SOURCE"); + + builder.addSink("SINK", "topic-sink", new StringSerializer(), new IntegerSerializer(), "PROCESS"); + + KafkaStreaming streaming = new KafkaStreaming(builder, config); + streaming.start(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/examples/WallclockTimestampExtractor.java b/streams/src/main/java/org/apache/kafka/streams/examples/WallclockTimestampExtractor.java new file mode 100644 index 0000000000000..26281d69d0293 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/examples/WallclockTimestampExtractor.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.streams.examples; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.streams.processor.TimestampExtractor; + +public class WallclockTimestampExtractor implements TimestampExtractor { + @Override + public long extract(ConsumerRecord record) { + return System.currentTimeMillis(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java new file mode 100644 index 0000000000000..915cf1c19ceaf --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.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.streams.kstream; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +/** + * KStream is an abstraction of a stream of key-value pairs. + * + * @param the type of keys + * @param the type of values + */ +public interface KStream { + + /** + * Creates a new stream consists of all elements of this stream which satisfy a predicate + * + * @param predicate the instance of Predicate + * @return the stream with only those elements that satisfy the predicate + */ + KStream filter(Predicate predicate); + + /** + * Creates a new stream consists all elements of this stream which do not satisfy a predicate + * + * @param predicate the instance of Predicate + * @return the stream with only those elements that do not satisfy the predicate + */ + KStream filterOut(Predicate predicate); + + /** + * Creates a new stream by applying transforming each element in this stream into a different element in the new stream. + * + * @param mapper the instance of KeyValueMapper + * @param the key type of the new stream + * @param the value type of the new stream + * @return the mapped stream + */ + KStream map(KeyValueMapper> mapper); + + /** + * Creates a new stream by applying transforming each value in this stream into a different value in the new stream. + * + * @param mapper the instance of ValueMapper + * @param the value type of the new stream + * @return the mapped stream + */ + KStream mapValues(ValueMapper mapper); + + /** + * Creates a new stream by applying transforming each element in this stream into zero or more elements in the new stream. + * + * @param mapper the instance of KeyValueMapper + * @param the key type of the new stream + * @param the value type of the new stream + * @return the mapped stream + */ + KStream flatMap(KeyValueMapper>> mapper); + + /** + * Creates a new stream by applying transforming each value in this stream into zero or more values in the new stream. + * + * @param processor the instance of Processor + * @param the value type of the new stream + * @return the mapped stream + */ + KStream flatMapValues(ValueMapper> processor); + + /** + * Creates a new windowed stream using a specified window instance. + * + * @param windowDef the instance of Window + * @return the windowed stream + */ + KStreamWindowed with(WindowSupplier windowDef); + + /** + * Creates an array of streams from this stream. Each stream in the array corresponds to a predicate in + * supplied predicates in the same order. Predicates are evaluated in order. An element is streamed to + * a corresponding stream for the first predicate is evaluated true. + * An element will be dropped if none of the predicates evaluate true. + * + * @param predicates the ordered list of Predicate instances + * @return the new streams that each contain those elements for which their Predicate evaluated to true. + */ + KStream[] branch(Predicate... predicates); + + /** + * Sends key-value to a topic, also creates a new stream from the topic. + * This is equivalent to calling to(topic) and from(topic). + * + * @param topic the topic name + * @param the key type of the new stream + * @param the value type of the new stream + * @return the new stream that consumes the given topic + */ + KStream through(String topic); + + /** + * Sends key-value to a topic, also creates a new stream from the topic. + * This is equivalent to calling to(topic) and from(topic). + * + * @param topic the topic name + * @param keySerializer key serializer used to send key-value pairs, + * if not specified the default key serializer defined in the configuration will be used + * @param valSerializer value serializer used to send key-value pairs, + * if not specified the default value serializer defined in the configuration will be used + * @param keyDeserializer key deserializer used to create the new KStream, + * if not specified the default key deserializer defined in the configuration will be used + * @param valDeserializer value deserializer used to create the new KStream, + * if not specified the default value deserializer defined in the configuration will be used + * @param the key type of the new stream + * @param the value type of the new stream + * @return the new stream that consumes the given topic + */ + KStream through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer); + + /** + * Sends key-value to a topic using default serializers specified in the config. + * + * @param topic the topic name + */ + void to(String topic); + + /** + * Sends key-value to a topic. + * + * @param topic the topic name + * @param keySerializer key serializer used to send key-value pairs, + * if not specified the default serializer defined in the configs will be used + * @param valSerializer value serializer used to send key-value pairs, + * if not specified the default serializer defined in the configs will be used + */ + void to(String topic, Serializer keySerializer, Serializer valSerializer); + + /** + * Applies a stateful transformation to all elements in this stream. + * + * @param transformerSupplier the class of TransformerDef + * @return KStream + */ + KStream transform(TransformerSupplier> transformerSupplier); + + /** + * Applies a stateful transformation to all values in this stream. + * + * @param valueTransformerSupplier the class of TransformerDef + * @return KStream + */ + KStream transformValues(ValueTransformerSupplier valueTransformerSupplier); + + /** + * Processes all elements in this stream by applying a processor. + * + * @param processorSupplier the supplier of the Processor to use + * @return the new stream containing the processed output + */ + void process(ProcessorSupplier processorSupplier); + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java new file mode 100644 index 0000000000000..2d4dcc72f8057 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.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.streams.kstream; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.streams.kstream.internals.KStreamImpl; +import org.apache.kafka.streams.processor.TopologyBuilder; + +/** + * KStreamBuilder is the class to create KStream instances. + */ +public class KStreamBuilder extends TopologyBuilder { + + public KStreamBuilder() { + super(); + } + + /** + * Creates a KStream instance for the specified topic. The stream is added to the default synchronization group. + * The default deserializers specified in the config are used. + * + * @param topics the topic names, if empty default to all the topics in the config + * @return KStream + */ + public KStream from(String... topics) { + String name = KStreamImpl.SOURCE_NAME + KStreamImpl.INDEX.getAndIncrement(); + + addSource(name, topics); + + return new KStreamImpl<>(this, name); + } + + /** + * Creates a KStream instance for the specified topic. The stream is added to the default synchronization group. + * + * @param keyDeserializer key deserializer used to read this source KStream, + * if not specified the default deserializer defined in the configs will be used + * @param valDeserializer value deserializer used to read this source KStream, + * if not specified the default deserializer defined in the configs will be used + * @param topics the topic names, if empty default to all the topics in the config + * @return KStream + */ + public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + String name = KStreamImpl.SOURCE_NAME + KStreamImpl.INDEX.getAndIncrement(); + + addSource(name, keyDeserializer, valDeserializer, topics); + + return new KStreamImpl<>(this, name); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamWindowed.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamWindowed.java new file mode 100644 index 0000000000000..4d73128839f21 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamWindowed.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.streams.kstream; + +/** + * KStreamWindowed is an abstraction of a stream of key-value pairs with a window. + */ +public interface KStreamWindowed extends KStream { + + /** + * Creates a new stream by joining this windowed stream with the other windowed stream. + * Each element arrived from either of the streams is joined with elements in a window of each other. + * The resulting values are computed by applying a joiner. + * + * @param other the other windowed stream + * @param joiner ValueJoiner + * @param the value type of the other stream + * @param the value type of the new stream + * @return KStream + */ + KStream join(KStreamWindowed other, ValueJoiner joiner); + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValue.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValue.java new file mode 100644 index 0000000000000..f633f6e337532 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValue.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.streams.kstream; + +public class KeyValue { + + public final K key; + public final V value; + + public KeyValue(K key, V value) { + this.key = key; + this.value = value; + } + + public static KeyValue pair(K key, V value) { + return new KeyValue<>(key, value); + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java new file mode 100644 index 0000000000000..62b07f646c812 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.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.streams.kstream; + +public interface KeyValueMapper { + + R apply(K key, V value); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java new file mode 100644 index 0000000000000..c73622e460eca --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.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.streams.kstream; + +/** + * Represents a predicate (boolean-valued function) of two arguments. + * + * @param the type of key + * @param the type of value + */ +public interface Predicate { + + boolean test(K key, V value); + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindowSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindowSupplier.java new file mode 100644 index 0000000000000..0110c875a1419 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindowSupplier.java @@ -0,0 +1,265 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams.kstream; + +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.kstream.internals.FilteredIterator; +import org.apache.kafka.streams.kstream.internals.WindowSupport; +import org.apache.kafka.streams.processor.StateRestoreCallback; +import org.apache.kafka.streams.processor.internals.ProcessorContextImpl; +import org.apache.kafka.streams.processor.internals.RecordCollector; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.internals.Stamped; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.Map; + +public class SlidingWindowSupplier implements WindowSupplier { + private final String name; + private final long duration; + private final int maxCount; + private final Serializer keySerializer; + private final Serializer valueSerializer; + private final Deserializer keyDeserializer; + private final Deserializer valueDeserializer; + + public SlidingWindowSupplier( + String name, + long duration, + int maxCount, + Serializer keySerializer, + Serializer valueSerializer, + Deserializer keyDeseriaizer, + Deserializer valueDeserializer) { + this.name = name; + this.duration = duration; + this.maxCount = maxCount; + this.keySerializer = keySerializer; + this.valueSerializer = valueSerializer; + this.keyDeserializer = keyDeseriaizer; + this.valueDeserializer = valueDeserializer; + } + + @Override + public String name() { + return name; + } + + @Override + public Window get() { + return new SlidingWindow(); + } + + public class SlidingWindow extends WindowSupport implements Window { + private final Object lock = new Object(); + private ProcessorContext context; + private int slotNum; // used as a key for Kafka log compaction + private LinkedList list = new LinkedList(); + private HashMap> map = new HashMap<>(); + + @Override + public void init(ProcessorContext context) { + this.context = context; + SlidingWindowRegistryCallback restoreFunc = new SlidingWindowRegistryCallback(); + context.register(this, restoreFunc); + + for (ValueList valueList : map.values()) { + valueList.clearDirtyValues(); + } + this.slotNum = restoreFunc.slotNum; + } + + @Override + public Iterator findAfter(K key, final long timestamp) { + return find(key, timestamp, timestamp + duration); + } + + @Override + public Iterator findBefore(K key, final long timestamp) { + return find(key, timestamp - duration, timestamp); + } + + @Override + public Iterator find(K key, final long timestamp) { + return find(key, timestamp - duration, timestamp + duration); + } + + /* + * finds items in the window between startTime and endTime (both inclusive) + */ + private Iterator find(K key, final long startTime, final long endTime) { + final ValueList values = map.get(key); + + if (values == null) { + return Collections.emptyIterator(); + } else { + return new FilteredIterator>(values.iterator()) { + @Override + protected V filter(Value item) { + if (startTime <= item.timestamp && item.timestamp <= endTime) + return item.value; + else + return null; + } + }; + } + } + + @Override + public void put(K key, V value, long timestamp) { + synchronized (lock) { + slotNum++; + + list.offerLast(key); + + ValueList values = map.get(key); + if (values == null) { + values = new ValueList<>(); + map.put(key, values); + } + + values.add(slotNum, value, timestamp); + } + evictExcess(); + evictExpired(timestamp - duration); + } + + private void evictExcess() { + while (list.size() > maxCount) { + K oldestKey = list.pollFirst(); + + ValueList values = map.get(oldestKey); + values.removeFirst(); + + if (values.isEmpty()) map.remove(oldestKey); + } + } + + private void evictExpired(long cutoffTime) { + while (true) { + K oldestKey = list.peekFirst(); + + ValueList values = map.get(oldestKey); + Stamped oldestValue = values.first(); + + if (oldestValue.timestamp < cutoffTime) { + list.pollFirst(); + values.removeFirst(); + + if (values.isEmpty()) map.remove(oldestKey); + } else { + break; + } + } + } + + @Override + public String name() { + return name; + } + + @Override + public void flush() { + IntegerSerializer intSerializer = new IntegerSerializer(); + ByteArraySerializer byteArraySerializer = new ByteArraySerializer(); + + RecordCollector collector = ((ProcessorContextImpl) context).recordCollector(); + + for (Map.Entry> entry : map.entrySet()) { + ValueList values = entry.getValue(); + if (values.hasDirtyValues()) { + K key = entry.getKey(); + + byte[] keyBytes = keySerializer.serialize(name, key); + + Iterator> iterator = values.dirtyValueIterator(); + while (iterator.hasNext()) { + Value dirtyValue = iterator.next(); + byte[] slot = intSerializer.serialize("", dirtyValue.slotNum); + byte[] valBytes = valueSerializer.serialize(name, dirtyValue.value); + + byte[] combined = new byte[8 + 4 + keyBytes.length + 4 + valBytes.length]; + + int offset = 0; + offset += putLong(combined, offset, dirtyValue.timestamp); + offset += puts(combined, offset, keyBytes); + offset += puts(combined, offset, valBytes); + + if (offset != combined.length) + throw new IllegalStateException("serialized length does not match"); + + collector.send(new ProducerRecord<>(name, context.id(), slot, combined), byteArraySerializer, byteArraySerializer); + } + values.clearDirtyValues(); + } + } + } + + @Override + public void close() { + // TODO + } + + @Override + public boolean persistent() { + // TODO: should not be persistent, right? + return false; + } + + private class SlidingWindowRegistryCallback implements StateRestoreCallback { + + final IntegerDeserializer intDeserializer; + int slotNum = 0; + + SlidingWindowRegistryCallback() { + intDeserializer = new IntegerDeserializer(); + } + + @Override + public void restore(byte[] slot, byte[] bytes) { + + slotNum = intDeserializer.deserialize("", slot); + + int offset = 0; + // timestamp + long timestamp = getLong(bytes, offset); + offset += 8; + // key + int length = getInt(bytes, offset); + offset += 4; + K key = deserialize(bytes, offset, length, name, keyDeserializer); + offset += length; + // value + length = getInt(bytes, offset); + offset += 4; + V value = deserialize(bytes, offset, length, name, valueDeserializer); + + put(key, value, timestamp); + } + } + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java new file mode 100644 index 0000000000000..b67f619b9ac33 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.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.streams.kstream; + +import org.apache.kafka.streams.processor.ProcessorContext; + +public interface Transformer { + + /** + * Initialize this transformer with the given context. The framework ensures this is called once per processor when the topology + * that contains it is initialized. + *

                  + * If this tranformer is to be {@link #punctuate(long) called periodically} by the framework, then this method should + * {@link ProcessorContext#schedule(long) schedule itself} with the provided context. + * + * @param context the context; may not be null + */ + void init(ProcessorContext context); + + /** + * Transform the message with the given key and value. + * + * @param key the key for the message + * @param value the value for the message + * @return new value + */ + R transform(K key, V value); + + /** + * Perform any periodic operations, if this processor {@link ProcessorContext#schedule(long) schedule itself} with the context + * during {@link #init(ProcessorContext) initialization}. + * + * @param timestamp the stream time when this method is being called + */ + void punctuate(long timestamp); + + /** + * Close this processor and clean up any resources. + */ + void close(); + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/TransformerSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/TransformerSupplier.java new file mode 100644 index 0000000000000..2c2d8dd28c924 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/TransformerSupplier.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.streams.kstream; + +public interface TransformerSupplier { + + Transformer get(); + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java new file mode 100644 index 0000000000000..93fc359bcaa35 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.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.streams.kstream; + +public interface ValueJoiner { + + R apply(V1 value1, V2 value2); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java new file mode 100644 index 0000000000000..a32423d04926c --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.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.streams.kstream; + +public interface ValueMapper { + + V2 apply(V1 value); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java new file mode 100644 index 0000000000000..5b9e2ff2d1e4a --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java @@ -0,0 +1,56 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams.kstream; + +import org.apache.kafka.streams.processor.ProcessorContext; + +public interface ValueTransformer { + + /** + * Initialize this transformer with the given context. The framework ensures this is called once per processor when the topology + * that contains it is initialized. + *

                  + * If this tranformer is to be {@link #punctuate(long) called periodically} by the framework, then this method should + * {@link ProcessorContext#schedule(long) schedule itself} with the provided context. + * + * @param context the context; may not be null + */ + void init(ProcessorContext context); + + /** + * Transform the message with the given key and value. + * + * @param value the value for the message + * @return new value + */ + R transform(V value); + + /** + * Perform any periodic operations, if this processor {@link ProcessorContext#schedule(long) schedule itself} with the context + * during {@link #init(ProcessorContext) initialization}. + * + * @param timestamp the stream time when this method is being called + */ + void punctuate(long timestamp); + + /** + * Close this processor and clean up any resources. + */ + void close(); + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerSupplier.java new file mode 100644 index 0000000000000..5c053c714d465 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerSupplier.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.streams.kstream; + +public interface ValueTransformerSupplier { + + ValueTransformer get(); + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java new file mode 100644 index 0000000000000..a1456f6c421b6 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java @@ -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 org.apache.kafka.streams.kstream; + +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; + +import java.util.Iterator; + +public interface Window extends StateStore { + + void init(ProcessorContext context); + + Iterator find(K key, long timestamp); + + Iterator findAfter(K key, long timestamp); + + Iterator findBefore(K key, long timestamp); + + void put(K key, V value, long timestamp); +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/WindowSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/WindowSupplier.java new file mode 100644 index 0000000000000..46a2b9ee91622 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/WindowSupplier.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.streams.kstream; + +public interface WindowSupplier { + + String name(); + + Window get(); +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/FilteredIterator.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/FilteredIterator.java new file mode 100644 index 0000000000000..54d44f01b8723 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/FilteredIterator.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.streams.kstream.internals; + +import java.util.Iterator; + +public abstract class FilteredIterator implements Iterator { + + private Iterator inner; + private T nextValue = null; + + public FilteredIterator(Iterator inner) { + this.inner = inner; + + findNext(); + } + + @Override + public boolean hasNext() { + return nextValue != null; + } + + @Override + public T next() { + T value = nextValue; + findNext(); + + return value; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + private void findNext() { + while (inner.hasNext()) { + S item = inner.next(); + nextValue = filter(item); + if (nextValue != null) { + return; + } + } + nextValue = null; + } + + protected abstract T filter(S item); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamBranch.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamBranch.java new file mode 100644 index 0000000000000..06083b30b7d0a --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamBranch.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.streams.kstream.internals; + +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.kstream.Predicate; + +class KStreamBranch implements ProcessorSupplier { + + private final Predicate[] predicates; + + @SuppressWarnings("unchecked") + public KStreamBranch(Predicate ... predicates) { + this.predicates = predicates; + } + + @Override + public Processor get() { + return new KStreamBranchProcessor(); + } + + private class KStreamBranchProcessor extends AbstractProcessor { + @Override + public void process(K key, V value) { + for (int i = 0; i < predicates.length; i++) { + if (predicates[i].test(key, value)) { + // use forward with childIndex here and then break the loop + // so that no record is going to be piped to multiple streams + context().forward(key, value, i); + break; + } + } + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java new file mode 100644 index 0000000000000..0b1f1e05a4a47 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.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.streams.kstream.internals; + +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.kstream.Predicate; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +class KStreamFilter implements ProcessorSupplier { + + private final Predicate predicate; + private final boolean filterOut; + + public KStreamFilter(Predicate predicate, boolean filterOut) { + this.predicate = predicate; + this.filterOut = filterOut; + } + + @Override + public Processor get() { + return new KStreamFilterProcessor(); + } + + private class KStreamFilterProcessor extends AbstractProcessor { + @Override + public void process(K key, V value) { + if (filterOut ^ predicate.test(key, value)) { + context().forward(key, value); + } + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.java new file mode 100644 index 0000000000000..175a002d17e85 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.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.streams.kstream.internals; + +import org.apache.kafka.streams.kstream.KeyValue; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +class KStreamFlatMap implements ProcessorSupplier { + + private final KeyValueMapper>> mapper; + + KStreamFlatMap(KeyValueMapper>> mapper) { + this.mapper = mapper; + } + + @Override + public Processor get() { + return new KStreamFlatMapProcessor(); + } + + private class KStreamFlatMapProcessor extends AbstractProcessor { + @Override + public void process(K1 key, V1 value) { + for (KeyValue newPair : mapper.apply(key, value)) { + context().forward(newPair.key, newPair.value); + } + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java new file mode 100644 index 0000000000000..9b4559bd769c5 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.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.streams.kstream.internals; + +import org.apache.kafka.streams.kstream.ValueMapper; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +class KStreamFlatMapValues implements ProcessorSupplier { + + private final ValueMapper> mapper; + + KStreamFlatMapValues(ValueMapper> mapper) { + this.mapper = mapper; + } + + @Override + public Processor get() { + return new KStreamFlatMapValuesProcessor(); + } + + private class KStreamFlatMapValuesProcessor extends AbstractProcessor { + @Override + public void process(K1 key, V1 value) { + Iterable newValues = mapper.apply(value); + for (V2 v : newValues) { + context().forward(key, v); + } + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java new file mode 100644 index 0000000000000..8f56e0968b9d7 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -0,0 +1,223 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams.kstream.internals; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.kstream.KeyValue; +import org.apache.kafka.streams.kstream.TransformerSupplier; +import org.apache.kafka.streams.kstream.ValueTransformerSupplier; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamWindowed; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.Predicate; +import org.apache.kafka.streams.kstream.ValueMapper; +import org.apache.kafka.streams.kstream.WindowSupplier; +import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.TopologyBuilder; + +import java.lang.reflect.Array; +import java.util.concurrent.atomic.AtomicInteger; + +public class KStreamImpl implements KStream { + + private static final String FILTER_NAME = "KAFKA-FILTER-"; + + private static final String MAP_NAME = "KAFKA-MAP-"; + + private static final String MAPVALUES_NAME = "KAFKA-MAPVALUES-"; + + private static final String FLATMAP_NAME = "KAFKA-FLATMAP-"; + + private static final String FLATMAPVALUES_NAME = "KAFKA-FLATMAPVALUES-"; + + private static final String TRANSFORM_NAME = "KAFKA-TRANSFORM-"; + + private static final String TRANSFORMVALUES_NAME = "KAFKA-TRANSFORMVALUES-"; + + private static final String PROCESSOR_NAME = "KAFKA-PROCESSOR-"; + + private static final String BRANCH_NAME = "KAFKA-BRANCH-"; + + private static final String BRANCHCHILD_NAME = "KAFKA-BRANCHCHILD-"; + + private static final String WINDOWED_NAME = "KAFKA-WINDOWED-"; + + private static final String SINK_NAME = "KAFKA-SINK-"; + + public static final String JOINTHIS_NAME = "KAFKA-JOINTHIS-"; + + public static final String JOINOTHER_NAME = "KAFKA-JOINOTHER-"; + + public static final String JOINMERGE_NAME = "KAFKA-JOINMERGE-"; + + public static final String SOURCE_NAME = "KAFKA-SOURCE-"; + + public static final AtomicInteger INDEX = new AtomicInteger(1); + + protected final TopologyBuilder topology; + protected final String name; + + public KStreamImpl(TopologyBuilder topology, String name) { + this.topology = topology; + this.name = name; + } + + @Override + public KStream filter(Predicate predicate) { + String name = FILTER_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(name, new KStreamFilter<>(predicate, false), this.name); + + return new KStreamImpl<>(topology, name); + } + + @Override + public KStream filterOut(final Predicate predicate) { + String name = FILTER_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(name, new KStreamFilter<>(predicate, true), this.name); + + return new KStreamImpl<>(topology, name); + } + + @Override + public KStream map(KeyValueMapper> mapper) { + String name = MAP_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(name, new KStreamMap<>(mapper), this.name); + + return new KStreamImpl<>(topology, name); + } + + @Override + public KStream mapValues(ValueMapper mapper) { + String name = MAPVALUES_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(name, new KStreamMapValues<>(mapper), this.name); + + return new KStreamImpl<>(topology, name); + } + + @Override + public KStream flatMap(KeyValueMapper>> mapper) { + String name = FLATMAP_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(name, new KStreamFlatMap<>(mapper), this.name); + + return new KStreamImpl<>(topology, name); + } + + @Override + public KStream flatMapValues(ValueMapper> mapper) { + String name = FLATMAPVALUES_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(name, new KStreamFlatMapValues<>(mapper), this.name); + + return new KStreamImpl<>(topology, name); + } + + @Override + public KStreamWindowed with(WindowSupplier windowSupplier) { + String name = WINDOWED_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(name, new KStreamWindow<>(windowSupplier), this.name); + + return new KStreamWindowedImpl<>(topology, name, windowSupplier); + } + + @Override + @SuppressWarnings("unchecked") + public KStream[] branch(Predicate... predicates) { + String branchName = BRANCH_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(branchName, new KStreamBranch(predicates.clone()), this.name); + + KStream[] branchChildren = (KStream[]) Array.newInstance(KStream.class, predicates.length); + for (int i = 0; i < predicates.length; i++) { + String childName = BRANCHCHILD_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(childName, new KStreamPassThrough(), branchName); + + branchChildren[i] = new KStreamImpl<>(topology, childName); + } + + return branchChildren; + } + + @Override + public KStream through(String topic, + Serializer keySerializer, + Serializer valSerializer, + Deserializer keyDeserializer, + Deserializer valDeserializer) { + String sendName = SINK_NAME + INDEX.getAndIncrement(); + + topology.addSink(sendName, topic, keySerializer, valSerializer, this.name); + + String sourceName = SOURCE_NAME + INDEX.getAndIncrement(); + + topology.addSource(sourceName, keyDeserializer, valDeserializer, topic); + + return new KStreamImpl<>(topology, sourceName); + } + + @Override + public KStream through(String topic) { + return through(topic, (Serializer) null, (Serializer) null, (Deserializer) null, (Deserializer) null); + } + + @Override + public void to(String topic) { + String name = SINK_NAME + INDEX.getAndIncrement(); + + topology.addSink(name, topic, this.name); + } + + @Override + public void to(String topic, Serializer keySerializer, Serializer valSerializer) { + String name = SINK_NAME + INDEX.getAndIncrement(); + + topology.addSink(name, topic, keySerializer, valSerializer, this.name); + } + + @Override + public KStream transform(TransformerSupplier> transformerSupplier) { + String name = TRANSFORM_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(name, new KStreamTransform<>(transformerSupplier), this.name); + + return new KStreamImpl<>(topology, name); + } + + @Override + public KStream transformValues(ValueTransformerSupplier valueTransformerSupplier) { + String name = TRANSFORMVALUES_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(name, new KStreamTransformValues<>(valueTransformerSupplier), this.name); + + return new KStreamImpl<>(topology, name); + } + + @Override + public void process(final ProcessorSupplier processorSupplier) { + String name = PROCESSOR_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(name, processorSupplier, this.name); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoin.java new file mode 100644 index 0000000000000..997953f579d55 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoin.java @@ -0,0 +1,97 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.kstream.internals; + +import org.apache.kafka.streams.kstream.ValueJoiner; +import org.apache.kafka.streams.kstream.Window; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +import java.util.Iterator; + +class KStreamJoin implements ProcessorSupplier { + + private static abstract class Finder { + abstract Iterator find(K key, long timestamp); + } + + private final String windowName; + private final ValueJoiner joiner; + + KStreamJoin(String windowName, ValueJoiner joiner) { + this.windowName = windowName; + this.joiner = joiner; + } + + @Override + public Processor get() { + return new KStreamJoinProcessor(windowName); + } + + private class KStreamJoinProcessor extends AbstractProcessor { + + private final String windowName; + protected Finder finder; + + public KStreamJoinProcessor(String windowName) { + this.windowName = windowName; + } + + @SuppressWarnings("unchecked") + @Override + public void init(ProcessorContext context) { + super.init(context); + + // check if these two streams are joinable + if (!context.joinable()) + throw new IllegalStateException("Streams are not joinable."); + + final Window window = (Window) context.getStateStore(windowName); + + this.finder = new Finder() { + @Override + Iterator find(K key, long timestamp) { + return window.find(key, timestamp); + } + }; + } + + @Override + public void process(K key, V1 value) { + long timestamp = context().timestamp(); + Iterator iter = finder.find(key, timestamp); + if (iter != null) { + while (iter.hasNext()) { + context().forward(key, joiner.apply(value, iter.next())); + } + } + } + } + + public static ValueJoiner reverseJoiner(final ValueJoiner joiner) { + return new ValueJoiner() { + @Override + public R apply(T2 value2, T1 value1) { + return joiner.apply(value1, value2); + } + }; + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.java new file mode 100644 index 0000000000000..3868318f5d136 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.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.streams.kstream.internals; + +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.kstream.KeyValue; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +class KStreamMap implements ProcessorSupplier { + + private final KeyValueMapper> mapper; + + public KStreamMap(KeyValueMapper> mapper) { + this.mapper = mapper; + } + + @Override + public Processor get() { + return new KStreamMapProcessor(); + } + + private class KStreamMapProcessor extends AbstractProcessor { + @Override + public void process(K1 key, V1 value) { + KeyValue newPair = mapper.apply(key, value); + context().forward(newPair.key, newPair.value); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMapValues.java new file mode 100644 index 0000000000000..692b421599c06 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMapValues.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.streams.kstream.internals; + +import org.apache.kafka.streams.kstream.ValueMapper; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +class KStreamMapValues implements ProcessorSupplier { + + private final ValueMapper mapper; + + public KStreamMapValues(ValueMapper mapper) { + this.mapper = mapper; + } + + @Override + public Processor get() { + return new KStreamMapProcessor(); + } + + private class KStreamMapProcessor extends AbstractProcessor { + @Override + public void process(K1 key, V1 value) { + V2 newValue = mapper.apply(value); + context().forward(key, newValue); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.java new file mode 100644 index 0000000000000..59a815ba2b9b8 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.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.streams.kstream.internals; + +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +class KStreamPassThrough implements ProcessorSupplier { + + @Override + public Processor get() { + return new KStreamPassThroughProcessor(); + } + + public class KStreamPassThroughProcessor extends AbstractProcessor { + @Override + public void process(K key, V value) { + context().forward(key, value); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransform.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransform.java new file mode 100644 index 0000000000000..7ebab0eae4419 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransform.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.streams.kstream.internals; + +import org.apache.kafka.streams.kstream.KeyValue; +import org.apache.kafka.streams.kstream.Transformer; +import org.apache.kafka.streams.kstream.TransformerSupplier; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +public class KStreamTransform implements ProcessorSupplier { + + private final TransformerSupplier> transformerSupplier; + + public KStreamTransform(TransformerSupplier> transformerSupplier) { + this.transformerSupplier = transformerSupplier; + } + + @Override + public Processor get() { + return new KStreamTransformProcessor(transformerSupplier.get()); + } + + public static class KStreamTransformProcessor implements Processor { + + private final Transformer> transformer; + private ProcessorContext context; + + public KStreamTransformProcessor(Transformer> transformer) { + this.transformer = transformer; + } + + @Override + public void init(ProcessorContext context) { + transformer.init(context); + this.context = context; + } + + @Override + public void process(K1 key, V1 value) { + KeyValue pair = transformer.transform(key, value); + context.forward(pair.key, pair.value); + } + + @Override + public void punctuate(long timestamp) { + transformer.punctuate(timestamp); + } + + @Override + public void close() { + transformer.close(); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java new file mode 100644 index 0000000000000..6f989e6833cfc --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.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.streams.kstream.internals; + +import org.apache.kafka.streams.kstream.ValueTransformer; +import org.apache.kafka.streams.kstream.ValueTransformerSupplier; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +public class KStreamTransformValues implements ProcessorSupplier { + + private final ValueTransformerSupplier valueTransformerSupplier; + + public KStreamTransformValues(ValueTransformerSupplier valueTransformerSupplier) { + this.valueTransformerSupplier = valueTransformerSupplier; + } + + @Override + public Processor get() { + return new KStreamTransformValuesProcessor(valueTransformerSupplier.get()); + } + + public static class KStreamTransformValuesProcessor implements Processor { + + private final ValueTransformer valueTransformer; + private ProcessorContext context; + + public KStreamTransformValuesProcessor(ValueTransformer valueTransformer) { + this.valueTransformer = valueTransformer; + } + + @Override + public void init(ProcessorContext context) { + valueTransformer.init(context); + this.context = context; + } + + @Override + public void process(K key, V value) { + context.forward(key, valueTransformer.transform(value)); + } + + @Override + public void punctuate(long timestamp) { + valueTransformer.punctuate(timestamp); + } + + @Override + public void close() { + valueTransformer.close(); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindow.java new file mode 100644 index 0000000000000..29239364a8093 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindow.java @@ -0,0 +1,68 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.kstream.internals; + +import org.apache.kafka.streams.kstream.Window; +import org.apache.kafka.streams.kstream.WindowSupplier; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +public class KStreamWindow implements ProcessorSupplier { + + private final WindowSupplier windowSupplier; + + KStreamWindow(WindowSupplier windowSupplier) { + this.windowSupplier = windowSupplier; + } + + public WindowSupplier window() { + return windowSupplier; + } + + @Override + public Processor get() { + return new KStreamWindowProcessor(); + } + + private class KStreamWindowProcessor extends AbstractProcessor { + + private Window window; + + @Override + public void init(ProcessorContext context) { + super.init(context); + this.window = windowSupplier.get(); + this.window.init(context); + } + + @Override + public void process(K key, V value) { + synchronized (this) { + window.put(key, value, context().timestamp()); + context().forward(key, value); + } + } + + @Override + public void close() { + window.close(); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedImpl.java new file mode 100644 index 0000000000000..93160122e0ce7 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedImpl.java @@ -0,0 +1,54 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

                  + * http://www.apache.org/licenses/LICENSE-2.0 + *

                  + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams.kstream.internals; + +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamWindowed; +import org.apache.kafka.streams.kstream.ValueJoiner; +import org.apache.kafka.streams.kstream.WindowSupplier; +import org.apache.kafka.streams.processor.TopologyBuilder; + +public final class KStreamWindowedImpl extends KStreamImpl implements KStreamWindowed { + + private final WindowSupplier windowSupplier; + + public KStreamWindowedImpl(TopologyBuilder topology, String name, WindowSupplier windowSupplier) { + super(topology, name); + this.windowSupplier = windowSupplier; + } + + @Override + public KStream join(KStreamWindowed other, ValueJoiner valueJoiner) { + String thisWindowName = this.windowSupplier.name(); + String otherWindowName = ((KStreamWindowedImpl) other).windowSupplier.name(); + + KStreamJoin joinThis = new KStreamJoin<>(otherWindowName, valueJoiner); + KStreamJoin joinOther = new KStreamJoin<>(thisWindowName, KStreamJoin.reverseJoiner(valueJoiner)); + KStreamPassThrough joinMerge = new KStreamPassThrough<>(); + + String joinThisName = JOINTHIS_NAME + INDEX.getAndIncrement(); + String joinOtherName = JOINOTHER_NAME + INDEX.getAndIncrement(); + String joinMergeName = JOINMERGE_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(joinThisName, joinThis, this.name); + topology.addProcessor(joinOtherName, joinOther, ((KStreamImpl) other).name); + topology.addProcessor(joinMergeName, joinMerge, joinThisName, joinOtherName); + + return new KStreamImpl<>(topology, joinMergeName); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowSupport.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowSupport.java new file mode 100644 index 0000000000000..b54bcc998c539 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowSupport.java @@ -0,0 +1,159 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams.kstream.internals; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.streams.processor.internals.Stamped; + +import java.util.Iterator; + +public class WindowSupport { + + public static class ValueList { + Value head = null; + Value tail = null; + Value dirty = null; + + public void add(int slotNum, V value, long timestamp) { + Value v = new Value<>(slotNum, value, timestamp); + if (tail != null) { + tail.next = v; + } else { + head = v; + } + tail = v; + if (dirty == null) dirty = v; + } + + public Value first() { + return head; + } + + public void removeFirst() { + if (head != null) { + if (head == tail) tail = null; + head = head.next; + } + } + + public boolean isEmpty() { + return head == null; + } + + public boolean hasDirtyValues() { + return dirty != null; + } + + public void clearDirtyValues() { + dirty = null; + } + + public Iterator> iterator() { + return new ValueListIterator(head); + } + + public Iterator> dirtyValueIterator() { + return new ValueListIterator(dirty); + } + + } + + private static class ValueListIterator implements Iterator> { + + Value ptr; + + ValueListIterator(Value start) { + ptr = start; + } + + @Override + public boolean hasNext() { + return ptr != null; + } + + @Override + public Value next() { + Value value = ptr; + if (value != null) ptr = value.next; + return value; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + } + + public static class Value extends Stamped { + public final int slotNum; + private Value next = null; + + Value(int slotNum, V value, long timestamp) { + super(value, timestamp); + this.slotNum = slotNum; + } + } + + + public static long getLong(byte[] bytes, int offset) { + long value = 0; + for (int i = 0; i < 8; i++) { + value = (value << 8) | bytes[offset + i]; + } + return value; + } + + public static int getInt(byte[] bytes, int offset) { + int value = 0; + for (int i = 0; i < 4; i++) { + value = (value << 8) | bytes[offset + i]; + } + return value; + } + + public static int putLong(byte[] bytes, int offset, long value) { + for (int i = 7; i >= 0; i--) { + bytes[offset + i] = (byte) (value & 0xFF); + value = value >> 8; + } + return 8; + } + + public static int putInt(byte[] bytes, int offset, int value) { + for (int i = 3; i >= 0; i--) { + bytes[offset + i] = (byte) (value & 0xFF); + value = value >> 8; + } + return 4; + } + + public static int puts(byte[] bytes, int offset, byte[] value) { + offset += putInt(bytes, offset, value.length); + System.arraycopy(bytes, offset, value, 0, value.length); + return 4 + value.length; + } + + + public static T deserialize(byte[] bytes, int offset, int length, String topic, Deserializer deserializer) { + byte[] buf = new byte[length]; + System.arraycopy(bytes, offset, buf, 0, length); + return deserializer.deserialize(topic, buf); + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/AbstractProcessor.java b/streams/src/main/java/org/apache/kafka/streams/processor/AbstractProcessor.java new file mode 100644 index 0000000000000..01d0024d64d78 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/AbstractProcessor.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.streams.processor; + +/** + * An abstract implementation of {@link Processor} that manages the {@link ProcessorContext} instance and provides default no-op + * implementations of {@link #punctuate(long)} and {@link #close()}. + * + * @param the type of keys + * @param the type of values + */ +public abstract class AbstractProcessor implements Processor { + + private ProcessorContext context; + + protected AbstractProcessor() { + } + + @Override + public void init(ProcessorContext context) { + this.context = context; + } + + /** + * Perform any periodic operations, if this processor {@link ProcessorContext#schedule(long) schedule itself} with the context + * during {@link #init(ProcessorContext) initialization}. + *

                  + * This method does nothing by default; if desired, subclasses should override it with custom functionality. + *

                  + * + * @param streamTime the stream time when this method is being called + */ + @Override + public void punctuate(long streamTime) { + // do nothing + } + + /** + * Close this processor and clean up any resources. + *

                  + * This method does nothing by default; if desired, subclasses should override it with custom functionality. + *

                  + */ + @Override + public void close() { + // do nothing + } + + /** + * Get the processor's context set during {@link #init(ProcessorContext) initialization}. + * + * @return the processor context; null only when called prior to {@link #init(ProcessorContext) initialization}. + */ + protected final ProcessorContext context() { + return this.context; + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java new file mode 100644 index 0000000000000..3cade3a907424 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.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.streams.processor; + +/** + * A processor of messages. + * + * @param the type of keys + * @param the type of values + */ +public interface Processor { + + /** + * Initialize this processor with the given context. The framework ensures this is called once per processor when the topology + * that contains it is initialized. + *

                  + * If this processor is to be {@link #punctuate(long) called periodically} by the framework, then this method should + * {@link ProcessorContext#schedule(long) schedule itself} with the provided context. + * + * @param context the context; may not be null + */ + void init(ProcessorContext context); + + /** + * Process the message with the given key and value. + * + * @param key the key for the message + * @param value the value for the message + */ + void process(K key, V value); + + /** + * Perform any periodic operations, if this processor {@link ProcessorContext#schedule(long) schedule itself} with the context + * during {@link #init(ProcessorContext) initialization}. + * + * @param timestamp the stream time when this method is being called + */ + void punctuate(long timestamp); + + /** + * Close this processor and clean up any resources. + */ + void close(); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java new file mode 100644 index 0000000000000..adffe0ec8bf84 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java @@ -0,0 +1,106 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams.processor; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.StreamingMetrics; + +import java.io.File; + +public interface ProcessorContext { + + /** + * Returns the partition group id + * + * @return partition group id + */ + int id(); + + /** + * Returns the key serializer + * + * @return the key serializer + */ + Serializer keySerializer(); + + /** + * Returns the value serializer + * + * @return the value serializer + */ + Serializer valueSerializer(); + + /** + * Returns the key deserializer + * + * @return the key deserializer + */ + Deserializer keyDeserializer(); + + /** + * Returns the value deserializer + * + * @return the value deserializer + */ + Deserializer valueDeserializer(); + + /** + * Returns the state directory for the partition. + * + * @return the state directory + */ + File stateDir(); + + /** + * Returns Metrics instance + * + * @return StreamingMetrics + */ + StreamingMetrics metrics(); + + /** + * Check if this process's incoming streams are joinable + */ + boolean joinable(); + + /** + * Registers and possibly restores the specified storage engine. + * + * @param store the storage engine + */ + void register(StateStore store, StateRestoreCallback stateRestoreCallback); + + StateStore getStateStore(String name); + + void schedule(long interval); + + void forward(K key, V value); + + void forward(K key, V value, int childIndex); + + void commit(); + + String topic(); + + int partition(); + + long offset(); + + long timestamp(); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorSupplier.java new file mode 100644 index 0000000000000..719d3ac2a34ff --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorSupplier.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.streams.processor; + +public interface ProcessorSupplier { + + Processor get(); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/StateRestoreCallback.java b/streams/src/main/java/org/apache/kafka/streams/processor/StateRestoreCallback.java new file mode 100644 index 0000000000000..39decec6622e5 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/StateRestoreCallback.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.streams.processor; + +/** + * Restoration logic for log-backed state stores upon restart, + * it takes one record at a time from the logs to apply to the restoring state. + */ +public interface StateRestoreCallback { + + void restore(byte[] key, byte[] value); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java b/streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java new file mode 100644 index 0000000000000..38afe9bacad67 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/StateStore.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.streams.processor; + +/** + * A storage engine for managing state maintained by a stream processor. + * + *

                  + * This interface does not specify any query capabilities, which, of course, + * would be query engine specific. Instead it just specifies the minimum + * functionality required to reload a storage engine from its changelog as well + * as basic lifecycle management. + *

                  + */ +public interface StateStore { + + /** + * The name of this store. + * @return the storage name + */ + String name(); + + /** + * Flush any cached data + */ + void flush(); + + /** + * Close the storage engine + */ + void close(); + + /** + * If the storage is persistent + */ + boolean persistent(); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.java b/streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.java new file mode 100644 index 0000000000000..62098f21d56df --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.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.streams.processor; + +import org.apache.kafka.clients.consumer.ConsumerRecord; + +/** + * An interface that allows the KStream framework to extract a timestamp from a key-value pair + */ +public interface TimestampExtractor { + + /** + * Extracts a timestamp from a message + * + * @param record ConsumerRecord + * @return timestamp + */ + long extract(ConsumerRecord record); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java new file mode 100644 index 0000000000000..833e29b9d9eef --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java @@ -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 org.apache.kafka.streams.processor; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.processor.internals.ProcessorNode; +import org.apache.kafka.streams.processor.internals.ProcessorTopology; +import org.apache.kafka.streams.processor.internals.SinkNode; +import org.apache.kafka.streams.processor.internals.SourceNode; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * A component that is used to build a {@link ProcessorTopology}. A topology contains an acyclic graph of sources, processors, + * and sinks. A {@link SourceNode source} is a node in the graph that consumes one or more Kafka topics and forwards them to + * its child nodes. A {@link Processor processor} is a node in the graph that receives input messages from upstream nodes, + * processes that message, and optionally forwarding new messages to one or all of its children. Finally, a {@link SinkNode sink} + * is a node in the graph that receives messages from upstream nodes and writes them to a Kafka topic. This builder allows you + * to construct an acyclic graph of these nodes, and the builder is then passed into a new {@link KafkaStreaming} instance + * that will then {@link KafkaStreaming#start() begin consuming, processing, and producing messages}. + */ +public class TopologyBuilder { + + // list of node factories in a topological order + private ArrayList nodeFactories = new ArrayList<>(); + + private Set nodeNames = new HashSet<>(); + private Set sourceTopicNames = new HashSet<>(); + + private interface NodeFactory { + ProcessorNode build(); + } + + private class ProcessorNodeFactory implements NodeFactory { + public final String[] parents; + private final String name; + private final ProcessorSupplier supplier; + + public ProcessorNodeFactory(String name, String[] parents, ProcessorSupplier supplier) { + this.name = name; + this.parents = parents.clone(); + this.supplier = supplier; + } + + @Override + public ProcessorNode build() { + return new ProcessorNode(name, supplier.get()); + } + } + + private class SourceNodeFactory implements NodeFactory { + public final String[] topics; + private final String name; + private Deserializer keyDeserializer; + private Deserializer valDeserializer; + + private SourceNodeFactory(String name, String[] topics, Deserializer keyDeserializer, Deserializer valDeserializer) { + this.name = name; + this.topics = topics.clone(); + this.keyDeserializer = keyDeserializer; + this.valDeserializer = valDeserializer; + } + + @Override + public ProcessorNode build() { + return new SourceNode(name, keyDeserializer, valDeserializer); + } + } + + private class SinkNodeFactory implements NodeFactory { + public final String[] parents; + public final String topic; + private final String name; + private Serializer keySerializer; + private Serializer valSerializer; + + private SinkNodeFactory(String name, String[] parents, String topic, Serializer keySerializer, Serializer valSerializer) { + this.name = name; + this.parents = parents.clone(); + this.topic = topic; + this.keySerializer = keySerializer; + this.valSerializer = valSerializer; + } + @Override + public ProcessorNode build() { + return new SinkNode(name, topic, keySerializer, valSerializer); + } + } + + /** + * Create a new builder. + */ + public TopologyBuilder() {} + + /** + * Add a new source that consumes the named topics and forwards the messages to child processor and/or sink nodes. + * The source will use the {@link StreamingConfig#KEY_DESERIALIZER_CLASS_CONFIG default key deserializer} and + * {@link StreamingConfig#VALUE_DESERIALIZER_CLASS_CONFIG default value deserializer} specified in the + * {@link StreamingConfig streaming configuration}. + * + * @param name the unique name of the source used to reference this node when + * {@link #addProcessor(String, ProcessorSupplier, String...) adding processor children}. + * @param topics the name of one or more Kafka topics that this source is to consume + * @return this builder instance so methods can be chained together; never null + */ + public final TopologyBuilder addSource(String name, String... topics) { + return addSource(name, (Deserializer) null, (Deserializer) null, topics); + } + + /** + * Add a new source that consumes the named topics and forwards the messages to child processor and/or sink nodes. + * The sink will use the specified key and value deserializers. + * + * @param name the unique name of the source used to reference this node when + * {@link #addProcessor(String, ProcessorSupplier, String...) adding processor children}. + * @param keyDeserializer the {@link Deserializer key deserializer} used when consuming messages; may be null if the source + * should use the {@link StreamingConfig#KEY_DESERIALIZER_CLASS_CONFIG default key deserializer} specified in the + * {@link StreamingConfig streaming configuration} + * @param valDeserializer the {@link Deserializer value deserializer} used when consuming messages; may be null if the source + * should use the {@link StreamingConfig#VALUE_DESERIALIZER_CLASS_CONFIG default value deserializer} specified in the + * {@link StreamingConfig streaming configuration} + * @param topics the name of one or more Kafka topics that this source is to consume + * @return this builder instance so methods can be chained together; never null + */ + public final TopologyBuilder addSource(String name, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + if (nodeNames.contains(name)) + throw new TopologyException("Processor " + name + " is already added."); + + for (String topic : topics) { + if (sourceTopicNames.contains(topic)) + throw new TopologyException("Topic " + topic + " has already been registered by another source."); + + sourceTopicNames.add(topic); + } + + nodeNames.add(name); + nodeFactories.add(new SourceNodeFactory(name, topics, keyDeserializer, valDeserializer)); + return this; + } + + /** + * Add a new sink that forwards messages from upstream parent processor and/or source nodes to the named Kafka topic. + * The sink will use the {@link StreamingConfig#KEY_SERIALIZER_CLASS_CONFIG default key serializer} and + * {@link StreamingConfig#VALUE_SERIALIZER_CLASS_CONFIG default value serializer} specified in the + * {@link StreamingConfig streaming configuration}. + * + * @param name the unique name of the sink + * @param topic the name of the Kafka topic to which this sink should write its messages + * @return this builder instance so methods can be chained together; never null + */ + public final TopologyBuilder addSink(String name, String topic, String... parentNames) { + return addSink(name, topic, (Serializer) null, (Serializer) null, parentNames); + } + + /** + * Add a new sink that forwards messages from upstream parent processor and/or source nodes to the named Kafka topic. + * The sink will use the specified key and value serializers. + * + * @param name the unique name of the sink + * @param topic the name of the Kafka topic to which this sink should write its messages + * @param keySerializer the {@link Serializer key serializer} used when consuming messages; may be null if the sink + * should use the {@link StreamingConfig#KEY_SERIALIZER_CLASS_CONFIG default key serializer} specified in the + * {@link StreamingConfig streaming configuration} + * @param valSerializer the {@link Serializer value serializer} used when consuming messages; may be null if the sink + * should use the {@link StreamingConfig#VALUE_SERIALIZER_CLASS_CONFIG default value serializer} specified in the + * {@link StreamingConfig streaming configuration} + * @param parentNames the name of one or more source or processor nodes whose output message this sink should consume + * and write to its topic + * @return this builder instance so methods can be chained together; never null + */ + public final TopologyBuilder addSink(String name, String topic, Serializer keySerializer, Serializer valSerializer, String... parentNames) { + if (nodeNames.contains(name)) + throw new TopologyException("Processor " + name + " is already added."); + + if (parentNames != null) { + for (String parent : parentNames) { + if (parent.equals(name)) { + throw new TopologyException("Processor " + name + " cannot be a parent of itself."); + } + if (!nodeNames.contains(parent)) { + throw new TopologyException("Parent processor " + parent + " is not added yet."); + } + } + } + + nodeNames.add(name); + nodeFactories.add(new SinkNodeFactory(name, parentNames, topic, keySerializer, valSerializer)); + return this; + } + + /** + * Add a new processor node that receives and processes messages output by one or more parent source or processor node. + * Any new messages output by this processor will be forwarded to its child processor or sink nodes. + * @param name the unique name of the processor node + * @param supplier the supplier used to obtain this node's {@link Processor} instance + * @param parentNames the name of one or more source or processor nodes whose output messages this processor should receive + * and process + * @return this builder instance so methods can be chained together; never null + */ + public final TopologyBuilder addProcessor(String name, ProcessorSupplier supplier, String... parentNames) { + if (nodeNames.contains(name)) + throw new TopologyException("Processor " + name + " is already added."); + + if (parentNames != null) { + for (String parent : parentNames) { + if (parent.equals(name)) { + throw new TopologyException("Processor " + name + " cannot be a parent of itself."); + } + if (!nodeNames.contains(parent)) { + throw new TopologyException("Parent processor " + parent + " is not added yet."); + } + } + } + + nodeNames.add(name); + nodeFactories.add(new ProcessorNodeFactory(name, parentNames, supplier)); + return this; + } + + /** + * Build the topology. This is typically called automatically when passing this builder into the + * {@link KafkaStreaming#KafkaStreaming(TopologyBuilder, StreamingConfig)} constructor. + * + * @see KafkaStreaming#KafkaStreaming(TopologyBuilder, StreamingConfig) + */ + @SuppressWarnings("unchecked") + public ProcessorTopology build() { + List processorNodes = new ArrayList<>(nodeFactories.size()); + Map processorMap = new HashMap<>(); + Map topicSourceMap = new HashMap<>(); + + try { + // create processor nodes in a topological order ("nodeFactories" is already topologically sorted) + for (NodeFactory factory : nodeFactories) { + ProcessorNode node = factory.build(); + processorNodes.add(node); + processorMap.put(node.name(), node); + + if (factory instanceof ProcessorNodeFactory) { + for (String parent : ((ProcessorNodeFactory) factory).parents) { + processorMap.get(parent).addChild(node); + } + } else if (factory instanceof SourceNodeFactory) { + for (String topic : ((SourceNodeFactory) factory).topics) { + topicSourceMap.put(topic, (SourceNode) node); + } + } else if (factory instanceof SinkNodeFactory) { + for (String parent : ((SinkNodeFactory) factory).parents) { + processorMap.get(parent).addChild(node); + } + } else { + throw new TopologyException("Unknown definition class: " + factory.getClass().getName()); + } + } + } catch (Exception e) { + throw new KafkaException("ProcessorNode construction failed: this should not happen."); + } + + return new ProcessorTopology(processorNodes, topicSourceMap); + } + + /** + * Get the names of topics that are to be consumed by the source nodes created by this builder. + * @return the unmodifiable set of topic names used by source nodes, which changes as new sources are added; never null + */ + public Set sourceTopics() { + return Collections.unmodifiableSet(sourceTopicNames); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyException.java b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyException.java new file mode 100644 index 0000000000000..99d14058a1f7a --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyException.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.streams.processor; + +import org.apache.kafka.common.KafkaException; + +public class TopologyException extends KafkaException { + + private static final long serialVersionUID = 1L; + + public TopologyException(String message) { + super(message); + } + + public TopologyException(String name, Object value) { + this(name, value, null); + } + + public TopologyException(String name, Object value, String message) { + super("Invalid topology building" + (message == null ? "" : ": " + message)); + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java new file mode 100644 index 0000000000000..717df2c54abcd --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java @@ -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 org.apache.kafka.streams.processor.internals; + +import java.util.LinkedList; + +/** + * MinTimestampTracker implements {@link TimestampTracker} that maintains the min + * timestamp of the maintained stamped elements. + */ +public class MinTimestampTracker implements TimestampTracker { + + private final LinkedList> descendingSubsequence = new LinkedList<>(); + + // in the case that incoming traffic is very small, the records maybe put and polled + // within a single iteration, in this case we need to remember the last polled + // record's timestamp + private long lastKnownTime = NOT_KNOWN; + + public void addElement(Stamped elem) { + if (elem == null) throw new NullPointerException(); + + Stamped minElem = descendingSubsequence.peekLast(); + while (minElem != null && minElem.timestamp >= elem.timestamp) { + descendingSubsequence.removeLast(); + minElem = descendingSubsequence.peekLast(); + } + descendingSubsequence.offerLast(elem); + } + + public void removeElement(Stamped elem) { + if (elem != null && descendingSubsequence.peekFirst() == elem) + descendingSubsequence.removeFirst(); + + if (descendingSubsequence.isEmpty()) + lastKnownTime = elem.timestamp; + } + + public int size() { + return descendingSubsequence.size(); + } + + public long get() { + Stamped stamped = descendingSubsequence.peekFirst(); + + if (stamped == null) + return lastKnownTime; + else + return stamped.timestamp; + } + +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java new file mode 100644 index 0000000000000..44a6c5c1a2f3d --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.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.streams.processor.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.streams.processor.TimestampExtractor; + +import java.util.Collections; +import java.util.Comparator; +import java.util.Map; +import java.util.PriorityQueue; +import java.util.Set; + +/** + * A PartitionGroup is composed from a set of partitions. It also maintains the timestamp of this + * group, hence the associated task as the min timestamp across all partitions in the group. + */ +public class PartitionGroup { + + private final Map partitionQueues; + + private final PriorityQueue queuesByTime; + + private final TimestampExtractor timestampExtractor; + + public static class RecordInfo { + public RecordQueue queue; + + public ProcessorNode node() { + return queue.source(); + } + + public TopicPartition partition() { + return queue.partition(); + } + } + + // since task is thread-safe, we do not need to synchronize on local variables + private int totalBuffered; + + public PartitionGroup(Map partitionQueues, TimestampExtractor timestampExtractor) { + this.queuesByTime = new PriorityQueue<>(partitionQueues.size(), new Comparator() { + + @Override + public int compare(RecordQueue queue1, RecordQueue queue2) { + long time1 = queue1.timestamp(); + long time2 = queue2.timestamp(); + + if (time1 < time2) return -1; + if (time1 > time2) return 1; + return 0; + } + }); + + this.partitionQueues = partitionQueues; + + this.timestampExtractor = timestampExtractor; + + this.totalBuffered = 0; + } + + /** + * Get the next record and queue + * + * @return StampedRecord + */ + public StampedRecord nextRecord(RecordInfo info) { + StampedRecord record = null; + + RecordQueue queue = queuesByTime.poll(); + if (queue != null) { + // get the first record from this queue. + record = queue.poll(); + + if (queue.size() > 0) { + queuesByTime.offer(queue); + } + } + info.queue = queue; + + if (record != null) totalBuffered--; + + return record; + } + + /** + * Adds raw records to this partition group + * + * @param partition the partition + * @param rawRecords the raw records + * @return the queue size for the partition + */ + public int addRawRecords(TopicPartition partition, Iterable> rawRecords) { + RecordQueue recordQueue = partitionQueues.get(partition); + + int oldSize = recordQueue.size(); + int newSize = recordQueue.addRawRecords(rawRecords, timestampExtractor); + + // add this record queue to be considered for processing in the future if it was empty before + if (oldSize == 0 && newSize > 0) { + queuesByTime.offer(recordQueue); + } + + totalBuffered += newSize - oldSize; + + return newSize; + } + + public Set partitions() { + return Collections.unmodifiableSet(partitionQueues.keySet()); + } + + /** + * Return the timestamp of this partition group as the smallest + * partition timestamp among all its partitions + */ + public long timestamp() { + if (queuesByTime.isEmpty()) { + // if there is no data in all partitions, return the smallest of their last known times + long timestamp = Long.MAX_VALUE; + for (RecordQueue queue : partitionQueues.values()) { + if (timestamp > queue.timestamp()) + timestamp = queue.timestamp(); + } + return timestamp; + } else { + return queuesByTime.peek().timestamp(); + } + } + + public int numBuffered(TopicPartition partition) { + RecordQueue recordQueue = partitionQueues.get(partition); + + if (recordQueue == null) + throw new KafkaException("Record's partition does not belong to this partition-group."); + + return recordQueue.size(); + } + + public int numBuffered() { + return totalBuffered; + } + + public void close() { + queuesByTime.clear(); + partitionQueues.clear(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java new file mode 100644 index 0000000000000..60ac1df98b241 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -0,0 +1,214 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams.processor.internals; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.StreamingConfig; +import org.apache.kafka.streams.StreamingMetrics; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.StateRestoreCallback; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class ProcessorContextImpl implements ProcessorContext { + + private static final Logger log = LoggerFactory.getLogger(ProcessorContextImpl.class); + + private final int id; + private final StreamTask task; + private final StreamingMetrics metrics; + private final RecordCollector collector; + private final ProcessorStateManager stateMgr; + + private final Serializer keySerializer; + private final Serializer valSerializer; + private final Deserializer keyDeserializer; + private final Deserializer valDeserializer; + + private boolean initialized; + + @SuppressWarnings("unchecked") + public ProcessorContextImpl(int id, + StreamTask task, + StreamingConfig config, + RecordCollector collector, + ProcessorStateManager stateMgr, + StreamingMetrics metrics) { + this.id = id; + this.task = task; + this.metrics = metrics; + this.collector = collector; + this.stateMgr = stateMgr; + + this.keySerializer = config.getConfiguredInstance(StreamingConfig.KEY_SERIALIZER_CLASS_CONFIG, Serializer.class); + this.valSerializer = config.getConfiguredInstance(StreamingConfig.VALUE_SERIALIZER_CLASS_CONFIG, Serializer.class); + this.keyDeserializer = config.getConfiguredInstance(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG, Deserializer.class); + this.valDeserializer = config.getConfiguredInstance(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Deserializer.class); + + this.initialized = false; + } + + public RecordCollector recordCollector() { + return this.collector; + } + + public void initialized() { + this.initialized = true; + } + + @Override + public boolean joinable() { + Set partitions = this.task.partitions(); + Map> partitionsById = new HashMap<>(); + int firstId = -1; + for (TopicPartition partition : partitions) { + if (!partitionsById.containsKey(partition.partition())) { + partitionsById.put(partition.partition(), new ArrayList()); + } + partitionsById.get(partition.partition()).add(partition.topic()); + + if (firstId < 0) + firstId = partition.partition(); + } + + List topics = partitionsById.get(firstId); + for (List topicsPerPartition : partitionsById.values()) { + if (topics.size() != topicsPerPartition.size()) + return false; + + for (String topic : topicsPerPartition) { + if (!topics.contains(topic)) + return false; + } + } + + return true; + } + + @Override + public int id() { + return id; + } + + @Override + public Serializer keySerializer() { + return this.keySerializer; + } + + @Override + public Serializer valueSerializer() { + return this.valSerializer; + } + + @Override + public Deserializer keyDeserializer() { + return this.keyDeserializer; + } + + @Override + public Deserializer valueDeserializer() { + return this.valDeserializer; + } + + @Override + public File stateDir() { + return stateMgr.baseDir(); + } + + @Override + public StreamingMetrics metrics() { + return metrics; + } + + @Override + public void register(StateStore store, StateRestoreCallback stateRestoreCallback) { + if (initialized) + throw new KafkaException("Can only create state stores during initialization."); + + stateMgr.register(store, stateRestoreCallback); + } + + @Override + public StateStore getStateStore(String name) { + return stateMgr.getStore(name); + } + + @Override + public String topic() { + if (task.record() == null) + throw new IllegalStateException("this should not happen as topic() should only be called while a record is processed"); + + return task.record().topic(); + } + + @Override + public int partition() { + if (task.record() == null) + throw new IllegalStateException("this should not happen as partition() should only be called while a record is processed"); + + return task.record().partition(); + } + + @Override + public long offset() { + if (this.task.record() == null) + throw new IllegalStateException("this should not happen as offset() should only be called while a record is processed"); + + return this.task.record().offset(); + } + + @Override + public long timestamp() { + if (task.record() == null) + throw new IllegalStateException("this should not happen as timestamp() should only be called while a record is processed"); + + return task.record().timestamp; + } + + @Override + public void forward(K key, V value) { + task.forward(key, value); + } + + @Override + public void forward(K key, V value, int childIndex) { + task.forward(key, value, childIndex); + } + + @Override + public void commit() { + task.needCommit(); + } + + @Override + public void schedule(long interval) { + task.schedule(interval); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java new file mode 100644 index 0000000000000..9127c3f170e63 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.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.streams.processor.internals; + +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; + +import java.util.ArrayList; +import java.util.List; + +public class ProcessorNode { + + private final List> children; + + private final String name; + private final Processor processor; + + public ProcessorNode(String name) { + this(name, null); + } + + public ProcessorNode(String name, Processor processor) { + this.name = name; + this.processor = processor; + this.children = new ArrayList<>(); + } + + public final String name() { + return name; + } + + public final Processor processor() { + return processor; + } + + public final List> children() { + return children; + } + + public void addChild(ProcessorNode child) { + children.add(child); + } + + public void init(ProcessorContext context) { + processor.init(context); + } + + public void process(K key, V value) { + processor.process(key, value); + } + + public void close() { + processor.close(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java new file mode 100644 index 0000000000000..59a6394678f62 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -0,0 +1,231 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

                  + * http://www.apache.org/licenses/LICENSE-2.0 + *

                  + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams.processor.internals; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.streams.processor.StateRestoreCallback; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.streams.state.OffsetCheckpoint; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.nio.channels.FileChannel; +import java.nio.channels.FileLock; +import java.nio.channels.OverlappingFileLockException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +public class ProcessorStateManager { + + private static final Logger log = LoggerFactory.getLogger(ProcessorStateManager.class); + + public static final String CHECKPOINT_FILE_NAME = ".checkpoint"; + public static final String LOCK_FILE_NAME = ".lock"; + + private final int id; + private final File baseDir; + private final FileLock directoryLock; + private final Map stores; + private final Consumer restoreConsumer; + private final Map restoredOffsets; + private final Map checkpointedOffsets; + + public ProcessorStateManager(int id, File baseDir, Consumer restoreConsumer) throws IOException { + this.id = id; + this.baseDir = baseDir; + this.stores = new HashMap<>(); + this.restoreConsumer = restoreConsumer; + this.restoredOffsets = new HashMap<>(); + + // create the state directory for this task if missing (we won't create the parent directory) + createStateDirectory(baseDir); + + // try to acquire the exclusive lock on the state directory + directoryLock = lockStateDirectory(baseDir); + if (directoryLock == null) { + throw new IOException("Failed to lock the state directory: " + baseDir.getCanonicalPath()); + } + + // load the checkpoint information + OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); + this.checkpointedOffsets = new HashMap<>(checkpoint.read()); + + // delete the checkpoint file after finish loading its stored offsets + checkpoint.delete(); + } + + private static void createStateDirectory(File stateDir) throws IOException { + if (!stateDir.exists()) { + stateDir.mkdir(); + } + } + + public static FileLock lockStateDirectory(File stateDir) throws IOException { + File lockFile = new File(stateDir, ProcessorStateManager.LOCK_FILE_NAME); + FileChannel channel = new RandomAccessFile(lockFile, "rw").getChannel(); + try { + return channel.tryLock(); + } catch (OverlappingFileLockException e) { + return null; + } + } + + public File baseDir() { + return this.baseDir; + } + + public void register(StateStore store, StateRestoreCallback stateRestoreCallback) { + if (store.name().equals(CHECKPOINT_FILE_NAME)) + throw new IllegalArgumentException("Illegal store name: " + CHECKPOINT_FILE_NAME); + + if (this.stores.containsKey(store.name())) + throw new IllegalArgumentException("Store " + store.name() + " has already been registered."); + + // ---- register the store ---- // + + // check that the underlying change log topic exist or not + if (restoreConsumer.listTopics().containsKey(store.name())) { + boolean partitionNotFound = true; + for (PartitionInfo partitionInfo : restoreConsumer.partitionsFor(store.name())) { + if (partitionInfo.partition() == id) { + partitionNotFound = false; + break; + } + } + + if (partitionNotFound) + throw new IllegalStateException("Store " + store.name() + "'s change log does not contain the partition for group " + id); + + } else { + throw new IllegalStateException("Change log topic for store " + store.name() + " does not exist yet"); + } + + this.stores.put(store.name(), store); + + // ---- try to restore the state from change-log ---- // + + // subscribe to the store's partition + TopicPartition storePartition = new TopicPartition(store.name(), id); + if (!restoreConsumer.subscription().isEmpty()) { + throw new IllegalStateException("Restore consumer should have not subscribed to any partitions beforehand"); + } + restoreConsumer.assign(Collections.singletonList(storePartition)); + + // calculate the end offset of the partition + // TODO: this is a bit hacky to first seek then position to get the end offset + restoreConsumer.seekToEnd(storePartition); + long endOffset = restoreConsumer.position(storePartition); + + // restore from the checkpointed offset of the change log if it is persistent and the offset exists; + // restore the state from the beginning of the change log otherwise + if (checkpointedOffsets.containsKey(storePartition) && store.persistent()) { + restoreConsumer.seek(storePartition, checkpointedOffsets.get(storePartition)); + } else { + restoreConsumer.seekToBeginning(storePartition); + } + + // restore its state from changelog records; while restoring the log end offset + // should not change since it is only written by this thread. + while (true) { + for (ConsumerRecord record : restoreConsumer.poll(100).records(storePartition)) { + stateRestoreCallback.restore(record.key(), record.value()); + } + + if (restoreConsumer.position(storePartition) == endOffset) { + break; + } else if (restoreConsumer.position(storePartition) > endOffset) { + throw new IllegalStateException("Log end offset should not change while restoring"); + } + } + + // record the restored offset for its change log partition + long newOffset = restoreConsumer.position(storePartition); + restoredOffsets.put(storePartition, newOffset); + + // un-assign the change log partition + restoreConsumer.assign(Collections.emptyList()); + } + + public StateStore getStore(String name) { + return stores.get(name); + } + + public void cleanup() throws IOException { + // clean up any unknown files in the state directory + for (File file : this.baseDir.listFiles()) { + if (!this.stores.containsKey(file.getName())) { + log.info("Deleting state directory {}", file.getAbsolutePath()); + file.delete(); + } + } + } + + public void flush() { + if (!this.stores.isEmpty()) { + log.debug("Flushing stores."); + for (StateStore store : this.stores.values()) + store.flush(); + } + } + + public void close(Map ackedOffsets) throws IOException { + if (!stores.isEmpty()) { + log.debug("Closing stores."); + for (Map.Entry entry : stores.entrySet()) { + log.debug("Closing storage engine {}", entry.getKey()); + entry.getValue().flush(); + entry.getValue().close(); + } + + Map checkpointOffsets = new HashMap<>(); + for (String storeName : stores.keySet()) { + TopicPartition part = new TopicPartition(storeName, id); + + // only checkpoint the offset to the offsets file if it is persistent; + if (stores.get(storeName).persistent()) { + Long offset = ackedOffsets.get(part); + + if (offset == null) { + // if no record was produced. we need to check the restored offset. + offset = restoredOffsets.get(part); + } + + if (offset != null) { + // store the last offset + 1 (the log position after restoration) + checkpointOffsets.put(part, offset + 1); + } + } + } + + // write the checkpoint file before closing, to indicate clean shutdown + OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); + checkpoint.write(checkpointOffsets); + } + + // release the state directory directoryLock + directoryLock.release(); + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java new file mode 100644 index 0000000000000..3efae6518f63d --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.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.streams.processor.internals; + +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class ProcessorTopology { + + private final List processorNodes; + private final Map sourceByTopics; + + public ProcessorTopology(List processorNodes, + Map sourceByTopics) { + this.processorNodes = Collections.unmodifiableList(processorNodes); + this.sourceByTopics = Collections.unmodifiableMap(sourceByTopics); + } + + public Set sourceTopics() { + return sourceByTopics.keySet(); + } + + public SourceNode source(String topic) { + return sourceByTopics.get(topic); + } + + public Set sources() { + return new HashSet<>(sourceByTopics.values()); + } + + public List processors() { + return processorNodes; + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java new file mode 100644 index 0000000000000..b4b7afe4f03e3 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java @@ -0,0 +1,56 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams.processor.internals; + +import java.util.PriorityQueue; + +public class PunctuationQueue { + + private PriorityQueue pq = new PriorityQueue<>(); + + public void schedule(PunctuationSchedule sched) { + synchronized (pq) { + pq.add(sched); + } + } + + public void close() { + synchronized (pq) { + pq.clear(); + } + } + + public boolean mayPunctuate(long timestamp, Punctuator punctuator) { + synchronized (pq) { + boolean punctuated = false; + PunctuationSchedule top = pq.peek(); + while (top != null && top.timestamp <= timestamp) { + PunctuationSchedule sched = top; + pq.poll(); + punctuator.punctuate(sched.node(), timestamp); + pq.add(sched.next()); + punctuated = true; + + top = pq.peek(); + } + + return punctuated; + } + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java new file mode 100644 index 0000000000000..dc9a50d32c249 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.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.streams.processor.internals; + +public class PunctuationSchedule extends Stamped { + + final long interval; + + public PunctuationSchedule(ProcessorNode node, long interval) { + this(node, System.currentTimeMillis(), interval); + } + + public PunctuationSchedule(ProcessorNode node, long time, long interval) { + super(node, time + interval); + this.interval = interval; + } + + public ProcessorNode node() { + return value; + } + + public PunctuationSchedule next() { + return new PunctuationSchedule(value, timestamp, interval); + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Punctuator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Punctuator.java new file mode 100644 index 0000000000000..d99e2ae960415 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Punctuator.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.streams.processor.internals; + +public interface Punctuator { + + void punctuate(ProcessorNode node, long streamTime); + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java new file mode 100644 index 0000000000000..ad2f647a2ac3d --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.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.streams.processor.internals; + +import org.apache.kafka.clients.producer.Callback; +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.TopicPartition; +import org.apache.kafka.common.serialization.Serializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; + +public class RecordCollector { + + private static final Logger log = LoggerFactory.getLogger(RecordCollector.class); + + private final Producer producer; + private final Map offsets; + private final Callback callback = new Callback() { + public void onCompletion(RecordMetadata metadata, Exception exception) { + if (exception == null) { + TopicPartition tp = new TopicPartition(metadata.topic(), metadata.partition()); + offsets.put(tp, metadata.offset()); + } else { + log.error("Error sending record: ", exception); + } + } + }; + + + public RecordCollector(Producer producer) { + this.producer = producer; + this.offsets = new HashMap<>(); + } + + public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { + byte[] keyBytes = keySerializer.serialize(record.topic(), record.key()); + byte[] valBytes = valueSerializer.serialize(record.topic(), record.value()); + this.producer.send(new ProducerRecord<>(record.topic(), keyBytes, valBytes), callback); + } + + public void flush() { + this.producer.flush(); + } + + /** + * Closes this RecordCollector + */ + public void close() { + producer.close(); + } + + /** + * The last ack'd offset from the producer + * + * @return the map from TopicPartition to offset + */ + Map offsets() { + return this.offsets; + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java new file mode 100644 index 0000000000000..66f78d2150994 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java @@ -0,0 +1,140 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams.processor.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.streams.processor.TimestampExtractor; + +import java.util.ArrayDeque; + +/** + * RecordQueue is a FIFO queue of {@link StampedRecord} (ConsumerRecord + timestamp). It also keeps track of the + * partition timestamp defined as the minimum timestamp of records in its queue; in addition, its partition + * timestamp is monotonically increasing such that once it is advanced, it will not be decremented. + */ +public class RecordQueue { + + private final SourceNode source; + private final TopicPartition partition; + private final ArrayDeque fifoQueue; + private final TimestampTracker> timeTracker; + + private long partitionTime = TimestampTracker.NOT_KNOWN; + + public RecordQueue(TopicPartition partition, SourceNode source) { + this.partition = partition; + this.source = source; + + this.fifoQueue = new ArrayDeque<>(); + this.timeTracker = new MinTimestampTracker<>(); + } + + /** + * Returns the corresponding source node in the topology + * + * @return SourceNode + */ + public SourceNode source() { + return source; + } + + /** + * Returns the partition with which this queue is associated + * + * @return TopicPartition + */ + public TopicPartition partition() { + return partition; + } + + /** + * Add a batch of {@link ConsumerRecord} into the queue + * + * @param rawRecords the raw records + * @param timestampExtractor TimestampExtractor + * @return the size of this queue + */ + public int addRawRecords(Iterable> rawRecords, TimestampExtractor timestampExtractor) { + for (ConsumerRecord rawRecord : rawRecords) { + // deserialize the raw record, extract the timestamp and put into the queue + Object key = source.deserializeKey(rawRecord.topic(), rawRecord.key()); + Object value = source.deserializeValue(rawRecord.topic(), rawRecord.value()); + + ConsumerRecord record = new ConsumerRecord<>(rawRecord.topic(), rawRecord.partition(), rawRecord.offset(), key, value); + long timestamp = timestampExtractor.extract(record); + + StampedRecord stampedRecord = new StampedRecord(record, timestamp); + + fifoQueue.addLast(stampedRecord); + timeTracker.addElement(stampedRecord); + } + + return size(); + } + + /** + * Get the next {@link StampedRecord} from the queue + * + * @return StampedRecord + */ + public StampedRecord poll() { + StampedRecord elem = fifoQueue.pollFirst(); + + if (elem == null) + return null; + + timeTracker.removeElement(elem); + + // only advance the partition timestamp if its currently + // tracked min timestamp has exceeded its value + long timestamp = timeTracker.get(); + + if (timestamp > partitionTime) + partitionTime = timestamp; + + return elem; + } + + /** + * Returns the number of records in the queue + * + * @return the number of records + */ + public int size() { + return fifoQueue.size(); + } + + /** + * Tests if the queue is empty + * + * @return true if the queue is empty, otherwise false + */ + public boolean isEmpty() { + return fifoQueue.isEmpty(); + } + + /** + * Returns the tracked partition timestamp + * + * @return timestamp + */ + public long timestamp() { + return partitionTime; + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java new file mode 100644 index 0000000000000..e2d881c9cdea8 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.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.streams.processor.internals; + +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.processor.ProcessorContext; + +public class SinkNode extends ProcessorNode { + + private final String topic; + private Serializer keySerializer; + private Serializer valSerializer; + + private ProcessorContext context; + + public SinkNode(String name, String topic, Serializer keySerializer, Serializer valSerializer) { + super(name); + + this.topic = topic; + this.keySerializer = keySerializer; + this.valSerializer = valSerializer; + } + + @Override + public void addChild(ProcessorNode child) { + throw new UnsupportedOperationException("sink node does not allow addChild"); + } + + @SuppressWarnings("unchecked") + @Override + public void init(ProcessorContext context) { + this.context = context; + if (this.keySerializer == null) this.keySerializer = (Serializer) context.keySerializer(); + if (this.valSerializer == null) this.valSerializer = (Serializer) context.valueSerializer(); + } + + @Override + public void process(K key, V value) { + // send to all the registered topics + RecordCollector collector = ((ProcessorContextImpl) context).recordCollector(); + collector.send(new ProducerRecord<>(topic, key, value), keySerializer, valSerializer); + } + + @Override + public void close() { + // do nothing + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java new file mode 100644 index 0000000000000..fa4afafbbdf27 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.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.streams.processor.internals; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.streams.processor.ProcessorContext; + +public class SourceNode extends ProcessorNode { + + private Deserializer keyDeserializer; + private Deserializer valDeserializer; + private ProcessorContext context; + + public SourceNode(String name, Deserializer keyDeserializer, Deserializer valDeserializer) { + super(name); + + this.keyDeserializer = keyDeserializer; + this.valDeserializer = valDeserializer; + } + + public K deserializeKey(String topic, byte[] data) { + return keyDeserializer.deserialize(topic, data); + } + + public V deserializeValue(String topic, byte[] data) { + return valDeserializer.deserialize(topic, data); + } + + @SuppressWarnings("unchecked") + @Override + public void init(ProcessorContext context) { + this.context = context; + + // if serializers are null, get the default ones from the context + if (this.keyDeserializer == null) this.keyDeserializer = (Deserializer) context.keyDeserializer(); + if (this.valDeserializer == null) this.valDeserializer = (Deserializer) context.valueDeserializer(); + } + + @Override + public void process(K key, V value) { + context.forward(key, value); + } + + @Override + public void close() { + // do nothing + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Stamped.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Stamped.java new file mode 100644 index 0000000000000..4e44667397d06 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Stamped.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.streams.processor.internals; + +public class Stamped implements Comparable { + + public final V value; + public final long timestamp; + + public Stamped(V value, long timestamp) { + this.value = value; + this.timestamp = timestamp; + } + + @Override + public int compareTo(Object other) { + long otherTimestamp = ((Stamped) other).timestamp; + + if (timestamp < otherTimestamp) return -1; + else if (timestamp > otherTimestamp) return 1; + return 0; + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java new file mode 100644 index 0000000000000..febd9389fd44c --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.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.streams.processor.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; + +public class StampedRecord extends Stamped> { + + public StampedRecord(ConsumerRecord record, long timestamp) { + super(record, timestamp); + } + + public String topic() { + return value.topic(); + } + + public int partition() { + return value.partition(); + } + + public Object key() { + return value.key(); + } + + public Object value() { + return value.value(); + } + + public long offset() { + return value.offset(); + } + + @Override + public String toString() { + return value.toString() + ", timestamp = " + timestamp; + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java new file mode 100644 index 0000000000000..6afa427c5ece2 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -0,0 +1,354 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

                  + * http://www.apache.org/licenses/LICENSE-2.0 + *

                  + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams.processor.internals; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.streams.StreamingConfig; +import org.apache.kafka.streams.StreamingMetrics; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.TimestampExtractor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * A StreamTask is associated with a {@link PartitionGroup}, and is assigned to a StreamThread for processing. + */ +public class StreamTask implements Punctuator { + + private static final Logger log = LoggerFactory.getLogger(StreamTask.class); + + private final int id; + private final int maxBufferedSize; + + private final Consumer consumer; + private final PartitionGroup partitionGroup; + private final PartitionGroup.RecordInfo recordInfo = new PartitionGroup.RecordInfo(); + private final PunctuationQueue punctuationQueue; + private final ProcessorContextImpl processorContext; + private final ProcessorTopology topology; + + private final Map consumedOffsets; + private final RecordCollector recordCollector; + private final ProcessorStateManager stateMgr; + + private boolean commitRequested = false; + private boolean commitOffsetNeeded = false; + private StampedRecord currRecord = null; + private ProcessorNode currNode = null; + + /** + * Create {@link StreamTask} with its assigned partitions + * + * @param id the ID of this task + * @param consumer the instance of {@link Consumer} + * @param producer the instance of {@link Producer} + * @param restoreConsumer the instance of {@link Consumer} used when restoring state + * @param partitions the collection of assigned {@link TopicPartition} + * @param topology the instance of {@link ProcessorTopology} + * @param config the {@link StreamingConfig} specified by the user + * @param metrics the {@link StreamingMetrics} created by the thread + */ + public StreamTask(int id, + Consumer consumer, + Producer producer, + Consumer restoreConsumer, + Collection partitions, + ProcessorTopology topology, + StreamingConfig config, + StreamingMetrics metrics) { + + this.id = id; + this.consumer = consumer; + this.punctuationQueue = new PunctuationQueue(); + this.maxBufferedSize = config.getInt(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); + this.topology = topology; + + // create queues for each assigned partition and associate them + // to corresponding source nodes in the processor topology + Map partitionQueues = new HashMap<>(); + + for (TopicPartition partition : partitions) { + SourceNode source = topology.source(partition.topic()); + RecordQueue queue = createRecordQueue(partition, source); + partitionQueues.put(partition, queue); + } + + TimestampExtractor timestampExtractor = config.getConfiguredInstance(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, TimestampExtractor.class); + this.partitionGroup = new PartitionGroup(partitionQueues, timestampExtractor); + + // initialize the consumed and produced offset cache + this.consumedOffsets = new HashMap<>(); + + // create the record recordCollector that maintains the produced offsets + this.recordCollector = new RecordCollector(producer); + + log.info("Creating restoration consumer client for stream task [" + id + "]"); + + // create the processor state manager + try { + File stateFile = new File(config.getString(StreamingConfig.STATE_DIR_CONFIG), Integer.toString(id)); + this.stateMgr = new ProcessorStateManager(id, stateFile, restoreConsumer); + } catch (IOException e) { + throw new KafkaException("Error while creating the state manager", e); + } + + // initialize the topology with its own context + this.processorContext = new ProcessorContextImpl(id, this, config, recordCollector, stateMgr, metrics); + + // initialize the task by initializing all its processor nodes in the topology + for (ProcessorNode node : this.topology.processors()) { + this.currNode = node; + try { + node.init(this.processorContext); + } finally { + this.currNode = null; + } + } + + this.processorContext.initialized(); + } + + public int id() { + return id; + } + + public Set partitions() { + return this.partitionGroup.partitions(); + } + + /** + * Adds records to queues + * + * @param partition the partition + * @param records the records + */ + @SuppressWarnings("unchecked") + public void addRecords(TopicPartition partition, Iterable> records) { + int queueSize = partitionGroup.addRawRecords(partition, records); + + // if after adding these records, its partition queue's buffered size has been + // increased beyond the threshold, we can then pause the consumption for this partition + if (queueSize > this.maxBufferedSize) { + consumer.pause(partition); + } + } + + /** + * Process one record + * + * @return number of records left in the buffer of this task's partition group after the processing is done + */ + @SuppressWarnings("unchecked") + public int process() { + synchronized (this) { + // get the next record to process + StampedRecord record = partitionGroup.nextRecord(recordInfo); + + // if there is no record to process, return immediately + if (record == null) + return 0; + + try { + // process the record by passing to the source node of the topology + this.currRecord = record; + this.currNode = recordInfo.node(); + TopicPartition partition = recordInfo.partition(); + + log.debug("Start processing one record [" + currRecord + "]"); + + this.currNode.process(currRecord.key(), currRecord.value()); + + log.debug("Completed processing one record [" + currRecord + "]"); + + // update the consumed offset map after processing is done + consumedOffsets.put(partition, currRecord.offset()); + commitOffsetNeeded = true; + + // after processing this record, if its partition queue's buffered size has been + // decreased to the threshold, we can then resume the consumption on this partition + if (partitionGroup.numBuffered(partition) == this.maxBufferedSize) { + consumer.resume(partition); + } + } finally { + this.currRecord = null; + this.currNode = null; + } + + return partitionGroup.numBuffered(); + } + } + + /** + * Possibly trigger registered punctuation functions if + * current time has reached the defined stamp + * + * @param timestamp + */ + public boolean maybePunctuate(long timestamp) { + return punctuationQueue.mayPunctuate(timestamp, this); + } + + @Override + public void punctuate(ProcessorNode node, long timestamp) { + if (currNode != null) + throw new IllegalStateException("Current node is not null"); + + currNode = node; + try { + node.processor().punctuate(timestamp); + } finally { + currNode = null; + } + } + + public StampedRecord record() { + return this.currRecord; + } + + public ProcessorNode node() { + return this.currNode; + } + + public ProcessorTopology topology() { + return this.topology; + } + + /** + * Commit the current task state + */ + public void commit() { + // 1) flush produced records in the downstream and change logs of local states + recordCollector.flush(); + + // 2) flush local state + stateMgr.flush(); + + // 3) commit consumed offsets if it is dirty already + if (commitOffsetNeeded) { + Map consumedOffsetsAndMetadata = new HashMap<>(consumedOffsets.size()); + for (Map.Entry entry : consumedOffsets.entrySet()) { + consumedOffsetsAndMetadata.put(entry.getKey(), new OffsetAndMetadata(entry.getValue())); + } + consumer.commitSync(consumedOffsetsAndMetadata); + commitOffsetNeeded = false; + } + + commitRequested = false; + } + + /** + * Whether or not a request has been made to commit the current state + */ + public boolean commitNeeded() { + return this.commitRequested; + } + + /** + * Request committing the current task's state + */ + public void needCommit() { + this.commitRequested = true; + } + + /** + * Schedules a punctuation for the processor + * + * @param interval the interval in milliseconds + */ + public void schedule(long interval) { + if (currNode == null) + throw new IllegalStateException("Current node is null"); + + punctuationQueue.schedule(new PunctuationSchedule(currNode, interval)); + } + + public void close() { + this.partitionGroup.close(); + this.consumedOffsets.clear(); + + // close the processors + // make sure close() is called for each node even when there is a RuntimeException + RuntimeException exception = null; + for (ProcessorNode node : this.topology.processors()) { + currNode = node; + try { + node.close(); + } catch (RuntimeException e) { + exception = e; + } finally { + currNode = null; + } + } + + if (exception != null) + throw exception; + + try { + stateMgr.close(recordCollector.offsets()); + } catch (IOException e) { + throw new KafkaException("Error while closing the state manager in processor context", e); + } + } + + private RecordQueue createRecordQueue(TopicPartition partition, SourceNode source) { + return new RecordQueue(partition, source); + } + + @SuppressWarnings("unchecked") + public void forward(K key, V value) { + ProcessorNode thisNode = currNode; + for (ProcessorNode childNode : (List>) thisNode.children()) { + currNode = childNode; + try { + childNode.process(key, value); + } finally { + currNode = thisNode; + } + } + } + + @SuppressWarnings("unchecked") + public void forward(K key, V value, int childIndex) { + ProcessorNode thisNode = currNode; + ProcessorNode childNode = (ProcessorNode) thisNode.children().get(childIndex); + currNode = childNode; + try { + childNode.process(key, value); + } finally { + currNode = thisNode; + } + } + + public ProcessorContext context() { + return processorContext; + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java new file mode 100644 index 0000000000000..4a6833254d45d --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -0,0 +1,531 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

                  + * http://www.apache.org/licenses/LICENSE-2.0 + *

                  + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams.processor.internals; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.metrics.MeasurableStat; +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.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.StreamingConfig; +import org.apache.kafka.streams.StreamingMetrics; +import org.apache.kafka.streams.processor.TopologyBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.nio.channels.FileLock; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +public class StreamThread extends Thread { + + private static final Logger log = LoggerFactory.getLogger(StreamThread.class); + private static final AtomicInteger STREAMING_THREAD_ID_SEQUENCE = new AtomicInteger(1); + + private final AtomicBoolean running; + + protected final StreamingConfig config; + protected final TopologyBuilder builder; + protected final Producer producer; + protected final Consumer consumer; + protected final Consumer restoreConsumer; + + private final Map tasks; + private final String clientId; + private final Time time; + private final File stateDir; + private final long pollTimeMs; + private final long cleanTimeMs; + private final long commitTimeMs; + private final long totalRecordsToProcess; + private final StreamingMetricsImpl sensors; + + private long lastClean; + private long lastCommit; + private long recordsProcessed; + + final ConsumerRebalanceListener rebalanceListener = new ConsumerRebalanceListener() { + @Override + public void onPartitionsAssigned(Collection assignment) { + addPartitions(assignment); + lastClean = time.milliseconds(); // start the cleaning cycle + } + + @Override + public void onPartitionsRevoked(Collection assignment) { + commitAll(); + removePartitions(); + lastClean = Long.MAX_VALUE; // stop the cleaning cycle until partitions are assigned + } + }; + + public StreamThread(TopologyBuilder builder, + StreamingConfig config, + String clientId, + Metrics metrics, + Time time) throws Exception { + this(builder, config, null , null, null, clientId, metrics, time); + } + + StreamThread(TopologyBuilder builder, + StreamingConfig config, + Producer producer, + Consumer consumer, + Consumer restoreConsumer, + String clientId, + Metrics metrics, + Time time) throws Exception { + super("StreamThread-" + STREAMING_THREAD_ID_SEQUENCE.getAndIncrement()); + + this.config = config; + this.builder = builder; + this.clientId = clientId; + + // set the producer and consumer clients + this.producer = (producer != null) ? producer : createProducer(); + this.consumer = (consumer != null) ? consumer : createConsumer(); + this.restoreConsumer = (restoreConsumer != null) ? restoreConsumer : createRestoreConsumer(); + + // initialize the task list + this.tasks = new HashMap<>(); + + // read in task specific config values + this.stateDir = new File(this.config.getString(StreamingConfig.STATE_DIR_CONFIG)); + this.stateDir.mkdir(); + this.pollTimeMs = config.getLong(StreamingConfig.POLL_MS_CONFIG); + this.commitTimeMs = config.getLong(StreamingConfig.COMMIT_INTERVAL_MS_CONFIG); + this.cleanTimeMs = config.getLong(StreamingConfig.STATE_CLEANUP_DELAY_MS_CONFIG); + this.totalRecordsToProcess = config.getLong(StreamingConfig.TOTAL_RECORDS_TO_PROCESS); + + this.lastClean = Long.MAX_VALUE; // the cleaning cycle won't start until partition assignment + this.lastCommit = time.milliseconds(); + this.recordsProcessed = 0; + this.time = time; + + this.sensors = new StreamingMetricsImpl(metrics); + + this.running = new AtomicBoolean(true); + } + + private Producer createProducer() { + log.info("Creating producer client for stream thread [" + this.getName() + "]"); + return new KafkaProducer<>(config.getProducerConfigs(), + new ByteArraySerializer(), + new ByteArraySerializer()); + } + + private Consumer createConsumer() { + log.info("Creating consumer client for stream thread [" + this.getName() + "]"); + return new KafkaConsumer<>(config.getConsumerConfigs(), + new ByteArrayDeserializer(), + new ByteArrayDeserializer()); + } + + private Consumer createRestoreConsumer() { + log.info("Creating restore consumer client for stream thread [" + this.getName() + "]"); + return new KafkaConsumer<>(config.getConsumerConfigs(), + new ByteArrayDeserializer(), + new ByteArrayDeserializer()); + } + + /** + * Execute the stream processors + */ + @Override + public void run() { + log.info("Starting stream thread [" + this.getName() + "]"); + + try { + runLoop(); + } catch (RuntimeException e) { + log.error("Uncaught error during processing in thread [" + this.getName() + "]: ", e); + throw e; + } finally { + shutdown(); + } + } + + /** + * Shutdown this streaming thread. + */ + public void close() { + running.set(false); + } + + public Map tasks() { + return Collections.unmodifiableMap(tasks); + } + + private void shutdown() { + log.info("Shutting down stream thread [" + this.getName() + "]"); + + // Exceptions should not prevent this call from going through all shutdown steps. + try { + commitAll(); + } catch (Throwable e) { + // already logged in commitAll() + } + try { + producer.close(); + } catch (Throwable e) { + log.error("Failed to close producer in thread [" + this.getName() + "]: ", e); + } + try { + consumer.close(); + } catch (Throwable e) { + log.error("Failed to close consumer in thread [" + this.getName() + "]: ", e); + } + try { + restoreConsumer.close(); + } catch (Throwable e) { + log.error("Failed to close restore consumer in thread [" + this.getName() + "]: ", e); + } + try { + removePartitions(); + } catch (Throwable e) { + // already logged in removePartition() + } + + log.info("Stream thread shutdown complete [" + this.getName() + "]"); + } + + private void runLoop() { + try { + int totalNumBuffered = 0; + + consumer.subscribe(new ArrayList<>(builder.sourceTopics()), rebalanceListener); + + while (stillRunning()) { + long startPoll = time.milliseconds(); + + // try to fetch some records if necessary + ConsumerRecords records = consumer.poll(totalNumBuffered == 0 ? this.pollTimeMs : 0); + + if (!records.isEmpty()) { + for (StreamTask task : tasks.values()) { + for (TopicPartition partition : task.partitions()) { + task.addRecords(partition, records.records(partition)); + } + } + } + + long endPoll = time.milliseconds(); + sensors.pollTimeSensor.record(endPoll - startPoll); + + // try to process one record from each task + totalNumBuffered = 0; + + for (StreamTask task : tasks.values()) { + long startProcess = time.milliseconds(); + + totalNumBuffered += task.process(); + + sensors.processTimeSensor.record(time.milliseconds() - startProcess); + } + + maybePunctuate(); + maybeClean(); + maybeCommit(); + } + } catch (Exception e) { + throw new KafkaException(e); + } + } + + private boolean stillRunning() { + if (!running.get()) { + log.debug("Shutting down at user request."); + return false; + } + + if (totalRecordsToProcess >= 0 && recordsProcessed >= totalRecordsToProcess) { + log.debug("Shutting down as we've reached the user configured limit of {} records to process.", totalRecordsToProcess); + return false; + } + + return true; + } + + private void maybePunctuate() { + for (StreamTask task : tasks.values()) { + try { + long now = time.milliseconds(); + + if (task.maybePunctuate(now)) + sensors.punctuateTimeSensor.record(time.milliseconds() - now); + + } catch (Exception e) { + log.error("Failed to commit task #" + task.id() + " in thread [" + this.getName() + "]: ", e); + throw e; + } + } + } + + protected void maybeCommit() { + long now = time.milliseconds(); + + if (commitTimeMs >= 0 && lastCommit + commitTimeMs < now) { + log.trace("Committing processor instances because the commit interval has elapsed."); + + commitAll(); + lastCommit = now; + } else { + for (StreamTask task : tasks.values()) { + try { + if (task.commitNeeded()) + commitOne(task, time.milliseconds()); + } catch (Exception e) { + log.error("Failed to commit task #" + task.id() + " in thread [" + this.getName() + "]: ", e); + throw e; + } + } + } + } + + /** + * Commit the states of all its tasks + */ + private void commitAll() { + for (StreamTask task : tasks.values()) { + try { + commitOne(task, time.milliseconds()); + } catch (Exception e) { + log.error("Failed to commit task #" + task.id() + " in thread [" + this.getName() + "]: ", e); + throw e; + } + } + } + + /** + * Commit the state of a task + */ + private void commitOne(StreamTask task, long now) { + try { + task.commit(); + } catch (Exception e) { + log.error("Failed to commit task #" + task.id() + " in thread [" + this.getName() + "]: ", e); + throw e; + } + + sensors.commitTimeSensor.record(time.milliseconds() - now); + } + + /** + * Cleanup any states of the tasks that have been removed from this thread + */ + protected void maybeClean() { + long now = time.milliseconds(); + + if (now > lastClean + cleanTimeMs) { + File[] stateDirs = stateDir.listFiles(); + if (stateDirs != null) { + for (File dir : stateDirs) { + try { + Integer id = Integer.parseInt(dir.getName()); + + // try to acquire the exclusive lock on the state directory + FileLock directoryLock = null; + try { + directoryLock = ProcessorStateManager.lockStateDirectory(dir); + if (directoryLock != null) { + log.info("Deleting obsolete state directory {} after delayed {} ms.", dir.getAbsolutePath(), cleanTimeMs); + Utils.delete(dir); + } + } catch (IOException e) { + log.error("Failed to lock the state directory due to an unexpected exception", e); + } finally { + if (directoryLock != null) { + try { + directoryLock.release(); + } catch (IOException e) { + log.error("Failed to release the state directory lock"); + } + } + } + } catch (NumberFormatException e) { + // there may be some unknown files that sits in the same directory, + // we should ignore these files instead trying to delete them as well + } + } + } + + lastClean = now; + } + } + + protected StreamTask createStreamTask(int id, Collection partitionsForTask) { + sensors.taskCreationSensor.record(); + + return new StreamTask(id, consumer, producer, restoreConsumer, partitionsForTask, builder.build(), config, sensors); + } + + private void addPartitions(Collection assignment) { + HashSet partitions = new HashSet<>(assignment); + + // TODO: change this hard-coded co-partitioning behavior + for (TopicPartition partition : partitions) { + final Integer id = partition.partition(); + StreamTask task = tasks.get(id); + if (task == null) { + // get the partitions for the task + HashSet partitionsForTask = new HashSet<>(); + for (TopicPartition part : partitions) + if (part.partition() == id) + partitionsForTask.add(part); + + // create the task + try { + task = createStreamTask(id, partitionsForTask); + } catch (Exception e) { + log.error("Failed to create a task #" + id + " in thread [" + this.getName() + "]: ", e); + throw e; + } + tasks.put(id, task); + } + } + + lastClean = time.milliseconds(); + } + + private void removePartitions() { + + // TODO: change this clearing tasks behavior + for (StreamTask task : tasks.values()) { + log.info("Removing task {}", task.id()); + try { + task.close(); + } catch (Exception e) { + log.error("Failed to close a task #" + task.id() + " in thread [" + this.getName() + "]: ", e); + throw e; + } + sensors.taskDestructionSensor.record(); + } + tasks.clear(); + } + + private class StreamingMetricsImpl implements StreamingMetrics { + final Metrics metrics; + final String metricGrpName; + final Map metricTags; + + final Sensor commitTimeSensor; + final Sensor pollTimeSensor; + final Sensor processTimeSensor; + final Sensor punctuateTimeSensor; + final Sensor taskCreationSensor; + final Sensor taskDestructionSensor; + + public StreamingMetricsImpl(Metrics metrics) { + + this.metrics = metrics; + this.metricGrpName = "streaming-metrics"; + this.metricTags = new LinkedHashMap<>(); + this.metricTags.put("client-id", clientId + "-" + getName()); + + this.commitTimeSensor = metrics.sensor("commit-time"); + this.commitTimeSensor.add(new MetricName("commit-time-avg", metricGrpName, "The average commit time in ms", metricTags), new Avg()); + this.commitTimeSensor.add(new MetricName("commit-time-max", metricGrpName, "The maximum commit time in ms", metricTags), new Max()); + this.commitTimeSensor.add(new MetricName("commit-calls-rate", metricGrpName, "The average per-second number of commit calls", metricTags), new Rate(new Count())); + + this.pollTimeSensor = metrics.sensor("poll-time"); + this.pollTimeSensor.add(new MetricName("poll-time-avg", metricGrpName, "The average poll time in ms", metricTags), new Avg()); + this.pollTimeSensor.add(new MetricName("poll-time-max", metricGrpName, "The maximum poll time in ms", metricTags), new Max()); + this.pollTimeSensor.add(new MetricName("poll-calls-rate", metricGrpName, "The average per-second number of record-poll calls", metricTags), new Rate(new Count())); + + this.processTimeSensor = metrics.sensor("process-time"); + this.processTimeSensor.add(new MetricName("process-time-avg-ms", metricGrpName, "The average process time in ms", metricTags), new Avg()); + this.processTimeSensor.add(new MetricName("process-time-max-ms", metricGrpName, "The maximum process time in ms", metricTags), new Max()); + this.processTimeSensor.add(new MetricName("process-calls-rate", metricGrpName, "The average per-second number of process calls", metricTags), new Rate(new Count())); + + this.punctuateTimeSensor = metrics.sensor("punctuate-time"); + this.punctuateTimeSensor.add(new MetricName("punctuate-time-avg", metricGrpName, "The average punctuate time in ms", metricTags), new Avg()); + this.punctuateTimeSensor.add(new MetricName("punctuate-time-max", metricGrpName, "The maximum punctuate time in ms", metricTags), new Max()); + this.punctuateTimeSensor.add(new MetricName("punctuate-calls-rate", metricGrpName, "The average per-second number of punctuate calls", metricTags), new Rate(new Count())); + + this.taskCreationSensor = metrics.sensor("task-creation"); + this.taskCreationSensor.add(new MetricName("task-creation-rate", metricGrpName, "The average per-second number of newly created tasks", metricTags), new Rate(new Count())); + + this.taskDestructionSensor = metrics.sensor("task-destruction"); + this.taskDestructionSensor.add(new MetricName("task-destruction-rate", metricGrpName, "The average per-second number of destructed tasks", metricTags), new Rate(new Count())); + } + + @Override + public void recordLatency(Sensor sensor, long startNs, long endNs) { + sensor.record((endNs - startNs) / 1000000, endNs); + } + + @Override + public Sensor addLatencySensor(String scopeName, String entityName, String operationName, String... tags) { + // extract the additional tags if there are any + Map tagMap = new HashMap<>(this.metricTags); + if ((tags.length % 2) != 0) + throw new IllegalArgumentException("Tags needs to be specified in key-value pairs"); + + for (int i = 0; i < tags.length; i += 2) + tagMap.put(tags[i], tags[i + 1]); + + // first add the global operation metrics if not yet, with the global tags only + Sensor parent = metrics.sensor(operationName); + addLatencyMetrics(this.metricGrpName, parent, "all", operationName, this.metricTags); + + // add the store operation metrics with additional tags + Sensor sensor = metrics.sensor(entityName + "-" + operationName, parent); + addLatencyMetrics("streaming-" + scopeName + "-metrics", sensor, entityName, operationName, tagMap); + + return sensor; + } + + private void addLatencyMetrics(String metricGrpName, Sensor sensor, String entityName, String opName, Map tags) { + maybeAddMetric(sensor, new MetricName(opName + "-avg-latency-ms", metricGrpName, + "The average latency in milliseconds of " + entityName + " " + opName + " operation.", tags), new Avg()); + maybeAddMetric(sensor, new MetricName(opName + "-max-latency-ms", metricGrpName, + "The max latency in milliseconds of " + entityName + " " + opName + " operation.", tags), new Max()); + maybeAddMetric(sensor, new MetricName(opName + "-qps", metricGrpName, + "The average number of occurrence of " + entityName + " " + opName + " operation per second.", tags), new Rate(new Count())); + } + + private void maybeAddMetric(Sensor sensor, MetricName name, MeasurableStat stat) { + if (!metrics.metrics().containsKey(name)) + sensor.add(name, stat); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TimestampTracker.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TimestampTracker.java new file mode 100644 index 0000000000000..d8a012a1bdcba --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TimestampTracker.java @@ -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 org.apache.kafka.streams.processor.internals; + +/** + * TimestampTracker is a helper class for a sliding window implementation. + * It is assumed that stamped elements are added or removed in a FIFO manner. + * It maintains the timestamp, such as the min timestamp, the max timestamp, etc. + * of stamped elements that were added but not yet removed. + */ +public interface TimestampTracker { + + static final long NOT_KNOWN = -1L; + + /** + * Adds a stamped elements to this tracker. + * + * @param elem the added element + */ + void addElement(Stamped elem); + + /** + * Removed a stamped elements to this tracker. + * + * @param elem the removed element + */ + void removeElement(Stamped elem); + + /** + * Returns the current tracked timestamp + * + * @return timestamp, or {@link #NOT_KNOWN} when empty + */ + long get(); + + /** + * Returns the size of internal structure. The meaning of "size" depends on the implementation. + * + * @return size + */ + int size(); + +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/state/Entry.java b/streams/src/main/java/org/apache/kafka/streams/state/Entry.java new file mode 100644 index 0000000000000..183b691c1acf6 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/Entry.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.streams.state; + +public class Entry { + + private final K key; + private final V value; + + public Entry(K key, V value) { + this.key = key; + this.value = value; + } + + public K key() { + return key; + } + + public V value() { + return value; + } + + public String toString() { + return "Entry(" + key() + ", " + value() + ")"; + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/InMemoryKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/InMemoryKeyValueStore.java new file mode 100644 index 0000000000000..59a8496de2966 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/InMemoryKeyValueStore.java @@ -0,0 +1,140 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams.state; + +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.common.utils.Time; + +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.TreeMap; + +/** + * An in-memory key-value store based on a TreeMap + * + * @param The key type + * @param The value type + */ +public class InMemoryKeyValueStore extends MeteredKeyValueStore { + + public InMemoryKeyValueStore(String name, ProcessorContext context) { + this(name, context, new SystemTime()); + } + + public InMemoryKeyValueStore(String name, ProcessorContext context, Time time) { + super(name, new MemoryStore(name, context), context, "in-memory-state", time); + } + + private static class MemoryStore implements KeyValueStore { + + private final String name; + private final NavigableMap map; + private final ProcessorContext context; + + @SuppressWarnings("unchecked") + public MemoryStore(String name, ProcessorContext context) { + super(); + this.name = name; + this.map = new TreeMap<>(); + this.context = context; + } + + @Override + public String name() { + return this.name; + } + + @Override + public boolean persistent() { + return false; + } + + @Override + public V get(K key) { + return this.map.get(key); + } + + @Override + public void put(K key, V value) { + this.map.put(key, value); + } + + @Override + public void putAll(List> entries) { + for (Entry entry : entries) + put(entry.key(), entry.value()); + } + + @Override + public V delete(K key) { + return this.map.remove(key); + } + + @Override + public KeyValueIterator range(K from, K to) { + return new MemoryStoreIterator(this.map.subMap(from, true, to, false).entrySet().iterator()); + } + + @Override + public KeyValueIterator all() { + return new MemoryStoreIterator(this.map.entrySet().iterator()); + } + + @Override + public void flush() { + // do-nothing since it is in-memory + } + + @Override + public void close() { + // do-nothing + } + + private static class MemoryStoreIterator implements KeyValueIterator { + private final Iterator> iter; + + public MemoryStoreIterator(Iterator> iter) { + this.iter = iter; + } + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public Entry next() { + Map.Entry entry = iter.next(); + return new Entry<>(entry.getKey(), entry.getValue()); + } + + @Override + public void remove() { + iter.remove(); + } + + @Override + public void close() { + } + + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/KeyValueIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/KeyValueIterator.java new file mode 100644 index 0000000000000..0fbd4ae536f1f --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/KeyValueIterator.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.streams.state; + +import java.io.Closeable; +import java.util.Iterator; + +public interface KeyValueIterator extends Iterator>, Closeable { + + @Override + public void close(); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java new file mode 100644 index 0000000000000..e4faed1efc576 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java @@ -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 org.apache.kafka.streams.state; + +import org.apache.kafka.streams.processor.StateStore; + +import java.util.List; + +/** + * A key-value store that supports put/get/delete and range queries. + * + * @param The key type + * @param The value type + */ +public interface KeyValueStore extends StateStore { + + /** + * Get the value corresponding to this key + * + * @param key The key to fetch + * @return The value or null if no value is found. + * @throws NullPointerException If null is used for key. + */ + abstract public V get(K key); + + /** + * Update the value associated with this key + * + * @param key They key to associate the value to + * @param value The value + * @throws NullPointerException If null is used for key or value. + */ + abstract public void put(K key, V value); + + /** + * Update all the given key/value pairs + * + * @param entries A list of entries to put into the store. + * @throws NullPointerException If null is used for any key or value. + */ + abstract public void putAll(List> entries); + + /** + * Delete the value from the store (if there is one) + * + * @param key The key + * @return The old value or null if there is no such key. + * @throws NullPointerException If null is used for key. + */ + abstract public V delete(K key); + + /** + * Get an iterator over a given range of keys. This iterator MUST be closed after use. + * + * @param from The first key that could be in the range + * @param to The last key that could be in the range + * @return The iterator for this range. + * @throws NullPointerException If null is used for from or to. + */ + abstract public KeyValueIterator range(K from, K to); + + /** + * Return an iterator over all keys in the database. This iterator MUST be closed after use. + * + * @return An iterator of all key/value pairs in the store. + */ + abstract public KeyValueIterator all(); + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/MeteredKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/MeteredKeyValueStore.java new file mode 100644 index 0000000000000..68333d5c6bd7a --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/MeteredKeyValueStore.java @@ -0,0 +1,242 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams.state; + +import org.apache.kafka.streams.StreamingMetrics; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateRestoreCallback; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.processor.internals.ProcessorContextImpl; +import org.apache.kafka.streams.processor.internals.RecordCollector; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +public class MeteredKeyValueStore implements KeyValueStore { + + protected final KeyValueStore inner; + + private final Time time; + private final Sensor putTime; + private final Sensor getTime; + private final Sensor deleteTime; + private final Sensor putAllTime; + private final Sensor allTime; + private final Sensor rangeTime; + private final Sensor flushTime; + private final Sensor restoreTime; + private final StreamingMetrics metrics; + + private final String topic; + private final int partition; + private final Set dirty; + private final int maxDirty; + private final ProcessorContext context; + + // always wrap the logged store with the metered store + public MeteredKeyValueStore(final String name, final KeyValueStore inner, ProcessorContext context, String metricGrp, Time time) { + this.inner = inner; + + this.time = time; + this.metrics = context.metrics(); + this.putTime = this.metrics.addLatencySensor(metricGrp, name, "put", "store-name", name); + this.getTime = this.metrics.addLatencySensor(metricGrp, name, "get", "store-name", name); + this.deleteTime = this.metrics.addLatencySensor(metricGrp, name, "delete", "store-name", name); + this.putAllTime = this.metrics.addLatencySensor(metricGrp, name, "put-all", "store-name", name); + this.allTime = this.metrics.addLatencySensor(metricGrp, name, "all", "store-name", name); + this.rangeTime = this.metrics.addLatencySensor(metricGrp, name, "range", "store-name", name); + this.flushTime = this.metrics.addLatencySensor(metricGrp, name, "flush", "store-name", name); + this.restoreTime = this.metrics.addLatencySensor(metricGrp, name, "restore", "store-name", name); + + this.topic = name; + this.partition = context.id(); + + this.context = context; + + this.dirty = new HashSet(); + this.maxDirty = 100; // TODO: this needs to be configurable + + // register and possibly restore the state from the logs + long startNs = time.nanoseconds(); + try { + final Deserializer keyDeserializer = (Deserializer) context.keyDeserializer(); + final Deserializer valDeserializer = (Deserializer) context.valueDeserializer(); + + context.register(this, new StateRestoreCallback() { + @Override + public void restore(byte[] key, byte[] value) { + inner.put(keyDeserializer.deserialize(topic, key), + valDeserializer.deserialize(topic, value)); + } + }); + } finally { + this.metrics.recordLatency(this.restoreTime, startNs, time.nanoseconds()); + } + } + + @Override + public String name() { + return inner.name(); + } + + @Override + public boolean persistent() { + return inner.persistent(); + } + + @Override + public V get(K key) { + long startNs = time.nanoseconds(); + try { + return this.inner.get(key); + } finally { + this.metrics.recordLatency(this.getTime, startNs, time.nanoseconds()); + } + } + + @Override + public void put(K key, V value) { + long startNs = time.nanoseconds(); + try { + this.inner.put(key, value); + + this.dirty.add(key); + if (this.dirty.size() > this.maxDirty) + logChange(); + } finally { + this.metrics.recordLatency(this.putTime, startNs, time.nanoseconds()); + } + } + + @Override + public void putAll(List> entries) { + long startNs = time.nanoseconds(); + try { + this.inner.putAll(entries); + + for (Entry entry : entries) { + this.dirty.add(entry.key()); + } + + if (this.dirty.size() > this.maxDirty) + logChange(); + } finally { + this.metrics.recordLatency(this.putAllTime, startNs, time.nanoseconds()); + } + } + + @Override + public V delete(K key) { + long startNs = time.nanoseconds(); + try { + V value = this.inner.delete(key); + + this.dirty.add(key); + if (this.dirty.size() > this.maxDirty) + logChange(); + + return value; + } finally { + this.metrics.recordLatency(this.deleteTime, startNs, time.nanoseconds()); + } + } + + @Override + public KeyValueIterator range(K from, K to) { + return new MeteredKeyValueIterator(this.inner.range(from, to), this.rangeTime); + } + + @Override + public KeyValueIterator all() { + return new MeteredKeyValueIterator(this.inner.all(), this.allTime); + } + + @Override + public void close() { + inner.close(); + } + + @Override + public void flush() { + long startNs = time.nanoseconds(); + try { + this.inner.flush(); + logChange(); + } finally { + this.metrics.recordLatency(this.flushTime, startNs, time.nanoseconds()); + } + } + + private void logChange() { + RecordCollector collector = ((ProcessorContextImpl) context).recordCollector(); + Serializer keySerializer = (Serializer) context.keySerializer(); + Serializer valueSerializer = (Serializer) context.valueSerializer(); + + if (collector != null) { + for (K k : this.dirty) { + V v = this.inner.get(k); + collector.send(new ProducerRecord<>(this.topic, this.partition, k, v), keySerializer, valueSerializer); + } + this.dirty.clear(); + } + } + + private class MeteredKeyValueIterator implements KeyValueIterator { + + private final KeyValueIterator iter; + private final Sensor sensor; + private final long startNs; + + public MeteredKeyValueIterator(KeyValueIterator iter, Sensor sensor) { + this.iter = iter; + this.sensor = sensor; + this.startNs = time.nanoseconds(); + } + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public Entry next() { + return iter.next(); + } + + @Override + public void remove() { + iter.remove(); + } + + @Override + public void close() { + try { + iter.close(); + } finally { + metrics.recordLatency(this.sensor, this.startNs, time.nanoseconds()); + } + } + + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/OffsetCheckpoint.java b/streams/src/main/java/org/apache/kafka/streams/state/OffsetCheckpoint.java new file mode 100644 index 0000000000000..e04de68afde8c --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/OffsetCheckpoint.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.streams.state; + +import org.apache.kafka.common.TopicPartition; + +import java.io.BufferedReader; +import java.io.BufferedWriter; +import java.io.EOFException; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.FileReader; +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +/** + * This class saves out a map of topic/partition=>offsets to a file. The format of the file is UTF-8 text containing the following: + *

                  + *   <version>
                  + *   <n>
                  + *   <topic_name_1> <partition_1> <offset_1>
                  + *   .
                  + *   .
                  + *   .
                  + *   <topic_name_n> <partition_n> <offset_n>
                  + * 
                  + * The first line contains a number designating the format version (currently 0), the get line contains + * a number giving the total number of offsets. Each successive line gives a topic/partition/offset triple + * separated by spaces. + */ +public class OffsetCheckpoint { + + private static final int VERSION = 0; + + private final File file; + private final Object lock; + + public OffsetCheckpoint(File file) throws IOException { + new File(file + ".tmp").delete(); // try to delete any existing temp files for cleanliness + this.file = file; + this.lock = new Object(); + } + + public void write(Map offsets) throws IOException { + synchronized (lock) { + // write to temp file and then swap with the existing file + File temp = new File(file.getAbsolutePath() + ".tmp"); + + FileOutputStream fileOutputStream = new FileOutputStream(temp); + BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(fileOutputStream)); + try { + writeIntLine(writer, VERSION); + writeIntLine(writer, offsets.size()); + + // write the entries + for (Map.Entry entry : offsets.entrySet()) + writeEntry(writer, entry.getKey(), entry.getValue()); + + // flush the buffer and then fsync the underlying file + writer.flush(); + fileOutputStream.getFD().sync(); + } finally { + writer.close(); + } + + // swap new offset checkpoint file with previous one + if (!temp.renameTo(file)) { + // renameTo() fails on Windows if the destination file exists. + file.delete(); + if (!temp.renameTo(file)) + throw new IOException(String.format("File rename from %s to %s failed.", + temp.getAbsolutePath(), + file.getAbsolutePath())); + } + } + } + + private void writeIntLine(BufferedWriter writer, int number) throws IOException { + writer.write(Integer.toString(number)); + writer.newLine(); + } + + private void writeEntry(BufferedWriter writer, TopicPartition part, long offset) throws IOException { + writer.write(part.topic()); + writer.write(' '); + writer.write(Integer.toString(part.partition())); + writer.write(' '); + writer.write(Long.toString(offset)); + writer.newLine(); + } + + public Map read() throws IOException { + synchronized (lock) { + BufferedReader reader = null; + try { + reader = new BufferedReader(new FileReader(file)); + } catch (FileNotFoundException e) { + return Collections.emptyMap(); + } + + try { + int version = readInt(reader); + switch (version) { + case 0: + int expectedSize = readInt(reader); + Map offsets = new HashMap(); + String line = reader.readLine(); + while (line != null) { + String[] pieces = line.split("\\s+"); + if (pieces.length != 3) + throw new IOException(String.format("Malformed line in offset checkpoint file: '%s'.", + line)); + + String topic = pieces[0]; + int partition = Integer.parseInt(pieces[1]); + long offset = Long.parseLong(pieces[2]); + offsets.put(new TopicPartition(topic, partition), offset); + line = reader.readLine(); + } + if (offsets.size() != expectedSize) + throw new IOException(String.format("Expected %d entries but found only %d", + expectedSize, + offsets.size())); + return offsets; + + default: + throw new IllegalArgumentException("Unknown offset checkpoint version: " + version); + } + } finally { + if (reader != null) + reader.close(); + } + } + } + + private int readInt(BufferedReader reader) throws IOException { + String line = reader.readLine(); + if (line == null) + throw new EOFException("File ended prematurely."); + int val = Integer.parseInt(line); + return val; + } + + public void delete() throws IOException { + file.delete(); + } + + @Override + public String toString() { + return this.file.getAbsolutePath(); + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/RocksDBKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/RocksDBKeyValueStore.java new file mode 100644 index 0000000000000..373bba07b0442 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/RocksDBKeyValueStore.java @@ -0,0 +1,276 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams.state; + +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.utils.SystemTime; + +import org.apache.kafka.common.utils.Time; +import org.rocksdb.BlockBasedTableConfig; +import org.rocksdb.CompactionStyle; +import org.rocksdb.CompressionType; +import org.rocksdb.FlushOptions; +import org.rocksdb.Options; +import org.rocksdb.RocksDB; +import org.rocksdb.RocksDBException; +import org.rocksdb.RocksIterator; +import org.rocksdb.WriteOptions; + +import java.io.File; +import java.util.Comparator; +import java.util.List; +import java.util.NoSuchElementException; + +public class RocksDBKeyValueStore extends MeteredKeyValueStore { + + public RocksDBKeyValueStore(String name, ProcessorContext context) { + this(name, context, new SystemTime()); + } + + public RocksDBKeyValueStore(String name, ProcessorContext context, Time time) { + super(name, new RocksDBStore(name, context), context, "rocksdb-state", time); + } + + private static class RocksDBStore implements KeyValueStore { + + private static final int TTL_NOT_USED = -1; + + // TODO: these values should be configurable + private static final CompressionType COMPRESSION_TYPE = CompressionType.NO_COMPRESSION; + private static final CompactionStyle COMPACTION_STYLE = CompactionStyle.UNIVERSAL; + private static final long WRITE_BUFFER_SIZE = 32 * 1024 * 1024L; + private static final long BLOCK_CACHE_SIZE = 100 * 1024 * 1024L; + private static final long BLOCK_SIZE = 4096L; + private static final int TTL_SECONDS = TTL_NOT_USED; + private static final int MAX_WRITE_BUFFERS = 3; + private static final String DB_FILE_DIR = "rocksdb"; + + private final String topic; + private final int partition; + private final ProcessorContext context; + + private final Options options; + private final WriteOptions wOptions; + private final FlushOptions fOptions; + + private final String dbName; + private final String dirName; + + private RocksDB db; + + @SuppressWarnings("unchecked") + public RocksDBStore(String name, ProcessorContext context) { + this.topic = name; + this.partition = context.id(); + this.context = context; + + // initialize the rocksdb options + BlockBasedTableConfig tableConfig = new BlockBasedTableConfig(); + tableConfig.setBlockCacheSize(BLOCK_CACHE_SIZE); + tableConfig.setBlockSize(BLOCK_SIZE); + + options = new Options(); + options.setTableFormatConfig(tableConfig); + options.setWriteBufferSize(WRITE_BUFFER_SIZE); + options.setCompressionType(COMPRESSION_TYPE); + options.setCompactionStyle(COMPACTION_STYLE); + options.setMaxWriteBufferNumber(MAX_WRITE_BUFFERS); + options.setCreateIfMissing(true); + options.setErrorIfExists(false); + + wOptions = new WriteOptions(); + wOptions.setDisableWAL(true); + + fOptions = new FlushOptions(); + fOptions.setWaitForFlush(true); + + dbName = this.topic + "." + this.partition; + dirName = this.context.stateDir() + File.separator + DB_FILE_DIR; + + db = openDB(new File(dirName, dbName), this.options, TTL_SECONDS); + } + + private RocksDB openDB(File dir, Options options, int ttl) { + try { + if (ttl == TTL_NOT_USED) { + return RocksDB.open(options, dir.toString()); + } else { + throw new KafkaException("Change log is not supported for store " + this.topic + " since it is TTL based."); + // TODO: support TTL with change log? + // return TtlDB.open(options, dir.toString(), ttl, false); + } + } catch (RocksDBException e) { + // TODO: this needs to be handled more accurately + throw new KafkaException("Error opening store " + this.topic + " at location " + dir.toString(), e); + } + } + + @Override + public String name() { + return this.topic; + } + + @Override + public boolean persistent() { + return false; + } + + @Override + public byte[] get(byte[] key) { + try { + return this.db.get(key); + } catch (RocksDBException e) { + // TODO: this needs to be handled more accurately + throw new KafkaException("Error while executing get " + key.toString() + " from store " + this.topic, e); + } + } + + @Override + public void put(byte[] key, byte[] value) { + try { + if (value == null) { + db.remove(wOptions, key); + } else { + db.put(wOptions, key, value); + } + } catch (RocksDBException e) { + // TODO: this needs to be handled more accurately + throw new KafkaException("Error while executing put " + key.toString() + " from store " + this.topic, e); + } + } + + @Override + public void putAll(List> entries) { + for (Entry entry : entries) + put(entry.key(), entry.value()); + } + + @Override + public byte[] delete(byte[] key) { + byte[] value = get(key); + put(key, null); + return value; + } + + @Override + public KeyValueIterator range(byte[] from, byte[] to) { + return new RocksDBRangeIterator(db.newIterator(), from, to); + } + + @Override + public KeyValueIterator all() { + RocksIterator innerIter = db.newIterator(); + innerIter.seekToFirst(); + return new RocksDbIterator(innerIter); + } + + @Override + public void flush() { + try { + db.flush(fOptions); + } catch (RocksDBException e) { + // TODO: this needs to be handled more accurately + throw new KafkaException("Error while executing flush from store " + this.topic, e); + } + } + + @Override + public void close() { + flush(); + db.close(); + } + + private static class RocksDbIterator implements KeyValueIterator { + private final RocksIterator iter; + + public RocksDbIterator(RocksIterator iter) { + this.iter = iter; + } + + protected byte[] peekKey() { + return this.getEntry().key(); + } + + protected Entry getEntry() { + return new Entry<>(iter.key(), iter.value()); + } + + @Override + public boolean hasNext() { + return iter.isValid(); + } + + @Override + public Entry next() { + if (!hasNext()) + throw new NoSuchElementException(); + + Entry entry = this.getEntry(); + iter.next(); + + return entry; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("RocksDB iterator does not support remove"); + } + + @Override + public void close() { + } + + } + + private static class LexicographicComparator implements Comparator { + + @Override + public int compare(byte[] left, byte[] right) { + for (int i = 0, j = 0; i < left.length && j < right.length; i++, j++) { + int leftByte = left[i] & 0xff; + int rightByte = right[j] & 0xff; + if (leftByte != rightByte) { + return leftByte - rightByte; + } + } + return left.length - right.length; + } + } + + private static class RocksDBRangeIterator extends RocksDbIterator { + // RocksDB's JNI interface does not expose getters/setters that allow the + // comparator to be pluggable, and the default is lexicographic, so it's + // safe to just force lexicographic comparator here for now. + private final Comparator comparator = new LexicographicComparator(); + byte[] to; + + public RocksDBRangeIterator(RocksIterator iter, byte[] from, byte[] to) { + super(iter); + iter.seek(from); + this.to = to; + } + + @Override + public boolean hasNext() { + return super.hasNext() && comparator.compare(super.peekKey(), this.to) < 0; + } + } + + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java new file mode 100644 index 0000000000000..49171e340b356 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.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.streams.kstream; + +import org.apache.kafka.streams.kstream.internals.KStreamImpl; +import org.apache.kafka.streams.processor.TopologyException; +import org.junit.Test; + +public class KStreamBuilderTest { + + @Test(expected = TopologyException.class) + public void testFrom() { + final KStreamBuilder builder = new KStreamBuilder(); + + builder.from("topic-1", "topic-2"); + + builder.addSource(KStreamImpl.SOURCE_NAME + KStreamImpl.INDEX.decrementAndGet(), "topic-3"); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/FilteredIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/FilteredIteratorTest.java new file mode 100644 index 0000000000000..405c7c93407e6 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/FilteredIteratorTest.java @@ -0,0 +1,94 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams.kstream.internals; + +import static org.junit.Assert.assertEquals; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; + +public class FilteredIteratorTest { + + @Test + public void testFiltering() { + List list = Arrays.asList(3, 1, 4, 1, 5, 9, 2, 6, 5, 3, 5); + + Iterator filtered = new FilteredIterator(list.iterator()) { + protected String filter(Integer i) { + if (i % 3 == 0) return i.toString(); + return null; + } + }; + + List expected = Arrays.asList("3", "9", "6", "3"); + List result = new ArrayList(); + + while (filtered.hasNext()) { + result.add(filtered.next()); + } + + assertEquals(expected, result); + } + + @Test + public void testEmptySource() { + List list = new ArrayList(); + + Iterator filtered = new FilteredIterator(list.iterator()) { + protected String filter(Integer i) { + if (i % 3 == 0) return i.toString(); + return null; + } + }; + + List expected = new ArrayList(); + List result = new ArrayList(); + + while (filtered.hasNext()) { + result.add(filtered.next()); + } + + assertEquals(expected, result); + } + + @Test + public void testNoMatch() { + List list = Arrays.asList(3, 1, 4, 1, 5, 9, 2, 6, 5, 3, 5); + + Iterator filtered = new FilteredIterator(list.iterator()) { + protected String filter(Integer i) { + if (i % 7 == 0) return i.toString(); + return null; + } + }; + + List expected = new ArrayList(); + List result = new ArrayList(); + + while (filtered.hasNext()) { + result.add(filtered.next()); + } + + assertEquals(expected, result); + } + +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java new file mode 100644 index 0000000000000..40eba2f103a5f --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.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.streams.kstream.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.Predicate; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockProcessorSupplier; +import org.junit.Test; + +import java.lang.reflect.Array; + +import static org.junit.Assert.assertEquals; + +public class KStreamBranchTest { + + private String topicName = "topic"; + + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + @SuppressWarnings("unchecked") + @Test + public void testKStreamBranch() { + KStreamBuilder builder = new KStreamBuilder(); + + Predicate isEven = new Predicate() { + @Override + public boolean test(Integer key, String value) { + return (key % 2) == 0; + } + }; + Predicate isMultipleOfThree = new Predicate() { + @Override + public boolean test(Integer key, String value) { + return (key % 3) == 0; + } + }; + Predicate isOdd = new Predicate() { + @Override + public boolean test(Integer key, String value) { + return (key % 2) != 0; + } + }; + + final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6}; + + KStream stream; + KStream[] branches; + MockProcessorSupplier[] processors; + + stream = builder.from(keyDeserializer, valDeserializer, topicName); + branches = stream.branch(isEven, isMultipleOfThree, isOdd); + + assertEquals(3, branches.length); + + processors = (MockProcessorSupplier[]) Array.newInstance(MockProcessorSupplier.class, branches.length); + for (int i = 0; i < branches.length; i++) { + processors[i] = new MockProcessorSupplier<>(); + branches[i].process(processors[i]); + } + + KStreamTestDriver driver = new KStreamTestDriver(builder); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); + } + + assertEquals(3, processors[0].processed.size()); + assertEquals(1, processors[1].processed.size()); + assertEquals(2, processors[2].processed.size()); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java new file mode 100644 index 0000000000000..d1e5d38a987f7 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java @@ -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 org.apache.kafka.streams.kstream.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.Predicate; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockProcessorSupplier; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class KStreamFilterTest { + + private String topicName = "topic"; + + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + private Predicate isMultipleOfThree = new Predicate() { + @Override + public boolean test(Integer key, String value) { + return (key % 3) == 0; + } + }; + + @Test + public void testFilter() { + KStreamBuilder builder = new KStreamBuilder(); + final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; + + KStream stream; + MockProcessorSupplier processor; + + processor = new MockProcessorSupplier<>(); + stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream.filter(isMultipleOfThree).process(processor); + + KStreamTestDriver driver = new KStreamTestDriver(builder); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); + } + + assertEquals(2, processor.processed.size()); + } + + @Test + public void testFilterOut() { + KStreamBuilder builder = new KStreamBuilder(); + final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; + + KStream stream; + MockProcessorSupplier processor; + + processor = new MockProcessorSupplier<>(); + stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream.filterOut(isMultipleOfThree).process(processor); + + KStreamTestDriver driver = new KStreamTestDriver(builder); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); + } + + assertEquals(5, processor.processed.size()); + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java new file mode 100644 index 0000000000000..61b5ccd5a1dc4 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.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.streams.kstream.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.KeyValue; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockProcessorSupplier; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +import java.util.ArrayList; + +public class KStreamFlatMapTest { + + private String topicName = "topic"; + + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + @Test + public void testFlatMap() { + KStreamBuilder builder = new KStreamBuilder(); + + KeyValueMapper>> mapper = + new KeyValueMapper>>() { + @Override + public Iterable> apply(Integer key, String value) { + ArrayList> result = new ArrayList<>(); + for (int i = 0; i < key; i++) { + result.add(KeyValue.pair(Integer.toString(key * 10 + i), value)); + } + return result; + } + }; + + final int[] expectedKeys = {0, 1, 2, 3}; + + KStream stream; + MockProcessorSupplier processor; + + processor = new MockProcessorSupplier<>(); + stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream.flatMap(mapper).process(processor); + + KStreamTestDriver driver = new KStreamTestDriver(builder); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); + } + + assertEquals(6, processor.processed.size()); + + String[] expected = {"10:V1", "20:V2", "21:V2", "30:V3", "31:V3", "32:V3"}; + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } + } + +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java new file mode 100644 index 0000000000000..66faf077885c7 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java @@ -0,0 +1,77 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams.kstream.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.ValueMapper; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockProcessorSupplier; +import org.junit.Test; + +import java.util.ArrayList; + +import static org.junit.Assert.assertEquals; + +public class KStreamFlatMapValuesTest { + + private String topicName = "topic"; + + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + @Test + public void testFlatMapValues() { + KStreamBuilder builder = new KStreamBuilder(); + + ValueMapper> mapper = + new ValueMapper>() { + @Override + public Iterable apply(String value) { + ArrayList result = new ArrayList(); + result.add(value.toLowerCase()); + result.add(value); + return result; + } + }; + + final int[] expectedKeys = {0, 1, 2, 3}; + + KStream stream; + MockProcessorSupplier processor; + + processor = new MockProcessorSupplier<>(); + stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream.flatMapValues(mapper).process(processor); + + KStreamTestDriver driver = new KStreamTestDriver(builder); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); + } + + assertEquals(8, processor.processed.size()); + + String[] expected = {"0:v0", "0:V0", "1:v1", "1:V1", "2:v2", "2:V2", "3:v3", "3:V3"}; + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java new file mode 100644 index 0000000000000..2db488c790b0c --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java @@ -0,0 +1,137 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams.kstream.internals; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.Predicate; +import org.apache.kafka.streams.kstream.ValueJoiner; +import org.apache.kafka.streams.kstream.ValueMapper; +import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.UnlimitedWindowDef; +import org.junit.Test; + +import java.util.Collections; + +import static org.junit.Assert.assertEquals; + + +public class KStreamImplTest { + + @Test + public void testNumProcesses() { + final Deserializer deserializer = new StringDeserializer(); + final KStreamBuilder builder = new KStreamBuilder(); + + KStream source1 = builder.from(deserializer, deserializer, "topic-1", "topic-2"); + + KStream source2 = builder.from(deserializer, deserializer, "topic-3", "topic-4"); + + KStream stream1 = + source1.filter(new Predicate() { + @Override + public boolean test(String key, String value) { + return true; + } + }).filterOut(new Predicate() { + @Override + public boolean test(String key, String value) { + return false; + } + }); + + KStream stream2 = stream1.mapValues(new ValueMapper() { + @Override + public Integer apply(String value) { + return new Integer(value); + } + }); + + KStream stream3 = source2.flatMapValues(new ValueMapper>() { + @Override + public Iterable apply(String value) { + return Collections.singletonList(new Integer(value)); + } + }); + + KStream[] streams2 = stream2.branch( + new Predicate() { + @Override + public boolean test(String key, Integer value) { + return (value % 2) == 0; + } + }, + new Predicate() { + @Override + public boolean test(String key, Integer value) { + return true; + } + } + ); + + KStream[] streams3 = stream3.branch( + new Predicate() { + @Override + public boolean test(String key, Integer value) { + return (value % 2) == 0; + } + }, + new Predicate() { + @Override + public boolean test(String key, Integer value) { + return true; + } + } + ); + + KStream stream4 = streams2[0].with(new UnlimitedWindowDef("window")) + .join(streams3[0].with(new UnlimitedWindowDef("window")), new ValueJoiner() { + @Override + public Integer apply(Integer value1, Integer value2) { + return value1 + value2; + } + }); + + KStream stream5 = streams2[1].with(new UnlimitedWindowDef("window")) + .join(streams3[1].with(new UnlimitedWindowDef("window")), new ValueJoiner() { + @Override + public Integer apply(Integer value1, Integer value2) { + return value1 + value2; + } + }); + + stream4.to("topic-5"); + + stream5.through("topic-6").process(new MockProcessorSupplier<>()); + + assertEquals(2 + // sources + 2 + // stream1 + 1 + // stream2 + 1 + // stream3 + 1 + 2 + // streams2 + 1 + 2 + // streams3 + 2 + 3 + // stream4 + 2 + 3 + // stream5 + 1 + // to + 2 + // through + 1, // process + builder.build().processors().size()); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamJoinTest.java new file mode 100644 index 0000000000000..58899faeda094 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamJoinTest.java @@ -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 org.apache.kafka.streams.kstream.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.KStreamWindowed; +import org.apache.kafka.streams.kstream.KeyValue; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.ValueJoiner; +import org.apache.kafka.streams.kstream.ValueMapper; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.UnlimitedWindowDef; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class KStreamJoinTest { + + private String topic1 = "topic1"; + private String topic2 = "topic2"; + + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + private ValueJoiner joiner = new ValueJoiner() { + @Override + public String apply(String value1, String value2) { + return value1 + "+" + value2; + } + }; + + private ValueMapper valueMapper = new ValueMapper() { + @Override + public String apply(String value) { + return "#" + value; + } + }; + + private ValueMapper> valueMapper2 = new ValueMapper>() { + @Override + public Iterable apply(String value) { + return (Iterable) Utils.mkSet(value); + } + }; + + private KeyValueMapper> keyValueMapper = + new KeyValueMapper>() { + @Override + public KeyValue apply(Integer key, String value) { + return KeyValue.pair(key, value); + } + }; + + KeyValueMapper>> keyValueMapper2 = + new KeyValueMapper>>() { + @Override + public KeyValue> apply(Integer key, String value) { + return KeyValue.pair(key, (Iterable) Utils.mkSet(value)); + } + }; + + + @Test + public void testJoin() { + KStreamBuilder builder = new KStreamBuilder(); + + final int[] expectedKeys = new int[]{0, 1, 2, 3}; + + KStream stream1; + KStream stream2; + KStreamWindowed windowed1; + KStreamWindowed windowed2; + MockProcessorSupplier processor; + String[] expected; + + processor = new MockProcessorSupplier<>(); + stream1 = builder.from(keyDeserializer, valDeserializer, topic1); + stream2 = builder.from(keyDeserializer, valDeserializer, topic2); + windowed1 = stream1.with(new UnlimitedWindowDef("window1")); + windowed2 = stream2.with(new UnlimitedWindowDef("window2")); + + windowed1.join(windowed2, joiner).process(processor); + + KStreamTestDriver driver = new KStreamTestDriver(builder); + driver.setTime(0L); + + // push two items to the main stream. the other stream's window is empty + + for (int i = 0; i < 2; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } + + assertEquals(0, processor.processed.size()); + + // push two items to the other stream. the main stream's window has two items + + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); + } + + assertEquals(2, processor.processed.size()); + + expected = new String[]{"0:X0+Y0", "1:X1+Y1"}; + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } + + processor.processed.clear(); + + // push all items to the main stream. this should produce two items. + + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } + + assertEquals(2, processor.processed.size()); + + expected = new String[]{"0:X0+Y0", "1:X1+Y1"}; + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } + + processor.processed.clear(); + + // there will be previous two items + all items in the main stream's window, thus two are duplicates. + + // push all items to the other stream. this should produce 6 items + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); + } + + assertEquals(6, processor.processed.size()); + + expected = new String[]{"0:X0+Y0", "0:X0+Y0", "1:X1+Y1", "1:X1+Y1", "2:X2+Y2", "3:X3+Y3"}; + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } + } + + // TODO: test for joinability +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java new file mode 100644 index 0000000000000..2ae8a97515dcb --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java @@ -0,0 +1,73 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.kstream.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.KeyValue; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockProcessorSupplier; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class KStreamMapTest { + + private String topicName = "topic"; + + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + @Test + public void testMap() { + KStreamBuilder builder = new KStreamBuilder(); + + KeyValueMapper> mapper = + new KeyValueMapper>() { + @Override + public KeyValue apply(Integer key, String value) { + return KeyValue.pair(value, key); + } + }; + + final int[] expectedKeys = new int[]{0, 1, 2, 3}; + + KStream stream; + MockProcessorSupplier processor; + + processor = new MockProcessorSupplier<>(); + stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream.map(mapper).process(processor); + + KStreamTestDriver driver = new KStreamTestDriver(builder); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); + } + + assertEquals(4, processor.processed.size()); + + String[] expected = new String[]{"V0:0", "V1:1", "V2:2", "V3:3"}; + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java new file mode 100644 index 0000000000000..f830c0010b338 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.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.streams.kstream.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.ValueMapper; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockProcessorSupplier; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class KStreamMapValuesTest { + + private String topicName = "topic"; + + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + @Test + public void testFlatMapValues() { + KStreamBuilder builder = new KStreamBuilder(); + + ValueMapper mapper = + new ValueMapper() { + @Override + public Integer apply(String value) { + return value.length(); + } + }; + + final int[] expectedKeys = {1, 10, 100, 1000}; + + KStream stream; + MockProcessorSupplier processor = new MockProcessorSupplier<>(); + stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream.mapValues(mapper).process(processor); + + KStreamTestDriver driver = new KStreamTestDriver(builder); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topicName, expectedKeys[i], Integer.toString(expectedKeys[i])); + } + + assertEquals(4, processor.processed.size()); + + String[] expected = {"1:1", "10:2", "100:3", "1000:4"}; + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } + } + +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java new file mode 100644 index 0000000000000..e397dd1fe20dd --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.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.streams.kstream.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.KeyValue; +import org.apache.kafka.streams.kstream.Transformer; +import org.apache.kafka.streams.kstream.TransformerSupplier; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockProcessorSupplier; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class KStreamTransformTest { + + private String topicName = "topic"; + + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private IntegerDeserializer valDeserializer = new IntegerDeserializer(); + + @Test + public void testTransform() { + KStreamBuilder builder = new KStreamBuilder(); + + TransformerSupplier> transformerSupplier = + new TransformerSupplier>() { + public Transformer> get() { + return new Transformer>() { + + private int total = 0; + + @Override + public void init(ProcessorContext context) { + } + + @Override + public KeyValue transform(Integer key, Integer value) { + total += value; + return KeyValue.pair(key * 2, total); + } + + @Override + public void punctuate(long timestamp) { + } + + @Override + public void close() { + } + }; + } + }; + + final int[] expectedKeys = {1, 10, 100, 1000}; + + KStream stream; + MockProcessorSupplier processor = new MockProcessorSupplier<>(); + stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream.transform(transformerSupplier).process(processor); + + KStreamTestDriver driver = new KStreamTestDriver(builder); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topicName, expectedKeys[i], expectedKeys[i] * 10); + } + + assertEquals(4, processor.processed.size()); + + String[] expected = {"2:10", "20:110", "200:1110", "2000:11110"}; + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } + } + +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java new file mode 100644 index 0000000000000..c5c9b39893c70 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java @@ -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 org.apache.kafka.streams.kstream.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.ValueTransformer; +import org.apache.kafka.streams.kstream.ValueTransformerSupplier; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockProcessorSupplier; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class KStreamTransformValuesTest { + + private String topicName = "topic"; + + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private IntegerDeserializer valDeserializer = new IntegerDeserializer(); + + @Test + public void testTransform() { + KStreamBuilder builder = new KStreamBuilder(); + + ValueTransformerSupplier valueTransformerSupplier = + new ValueTransformerSupplier() { + public ValueTransformer get() { + return new ValueTransformer() { + + private int total = 0; + + @Override + public void init(ProcessorContext context) { + } + + @Override + public Integer transform(Integer value) { + total += value; + return total; + } + + @Override + public void punctuate(long timestamp) { + } + + @Override + public void close() { + } + }; + } + }; + + final int[] expectedKeys = {1, 10, 100, 1000}; + + KStream stream; + MockProcessorSupplier processor = new MockProcessorSupplier<>(); + stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream.transformValues(valueTransformerSupplier).process(processor); + + KStreamTestDriver driver = new KStreamTestDriver(builder); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topicName, expectedKeys[i], expectedKeys[i] * 10); + } + + assertEquals(4, processor.processed.size()); + + String[] expected = {"1:10", "10:110", "100:1110", "1000:11110"}; + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } + } + +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedTest.java new file mode 100644 index 0000000000000..c3dc7e08129ae --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedTest.java @@ -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 org.apache.kafka.streams.kstream.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.Window; +import org.apache.kafka.streams.kstream.WindowSupplier; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.UnlimitedWindowDef; +import org.junit.Test; + +import java.util.Iterator; + +import static org.junit.Assert.assertEquals; + +public class KStreamWindowedTest { + + private String topicName = "topic"; + private String windowName = "MyWindow"; + + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + @Test + public void testWindowedStream() { + KStreamBuilder builder = new KStreamBuilder(); + + final int[] expectedKeys = new int[]{0, 1, 2, 3}; + + KStream stream; + WindowSupplier windowSupplier; + + windowSupplier = new UnlimitedWindowDef<>(windowName); + stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream.with(windowSupplier); + + KStreamTestDriver driver = new KStreamTestDriver(builder); + Window window = (Window) driver.getStateStore(windowName); + driver.setTime(0L); + + // two items in the window + + for (int i = 0; i < 2; i++) { + driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); + } + + assertEquals(1, countItem(window.find(0, 0L))); + assertEquals(1, countItem(window.find(1, 0L))); + assertEquals(0, countItem(window.find(2, 0L))); + assertEquals(0, countItem(window.find(3, 0L))); + + // previous two items + all items, thus two are duplicates, in the window + + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topicName, expectedKeys[i], "Y" + expectedKeys[i]); + } + + assertEquals(2, countItem(window.find(0, 0L))); + assertEquals(2, countItem(window.find(1, 0L))); + assertEquals(1, countItem(window.find(2, 0L))); + assertEquals(1, countItem(window.find(3, 0L))); + } + + + private int countItem(Iterator iter) { + int i = 0; + while (iter.hasNext()) { + i++; + iter.next(); + } + return i; + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java new file mode 100644 index 0000000000000..00522d53f6167 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java @@ -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 org.apache.kafka.streams.processor; + +import static org.junit.Assert.assertEquals; + +import org.apache.kafka.test.MockProcessorSupplier; +import org.junit.Test; + +public class TopologyBuilderTest { + + @Test(expected = TopologyException.class) + public void testAddSourceWithSameName() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addSource("source", "topic-1"); + builder.addSource("source", "topic-2"); + } + + @Test(expected = TopologyException.class) + public void testAddSourceWithSameTopic() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addSource("source", "topic-1"); + builder.addSource("source-2", "topic-1"); + } + + @Test(expected = TopologyException.class) + public void testAddProcessorWithSameName() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addSource("source", "topic-1"); + builder.addProcessor("processor", new MockProcessorSupplier(), "source"); + builder.addProcessor("processor", new MockProcessorSupplier(), "source"); + } + + @Test(expected = TopologyException.class) + public void testAddProcessorWithWrongParent() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addProcessor("processor", new MockProcessorSupplier(), "source"); + } + + @Test(expected = TopologyException.class) + public void testAddProcessorWithSelfParent() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addProcessor("processor", new MockProcessorSupplier(), "processor"); + } + + @Test(expected = TopologyException.class) + public void testAddSinkWithSameName() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addSource("source", "topic-1"); + builder.addSink("sink", "topic-2", "source"); + builder.addSink("sink", "topic-3", "source"); + } + + @Test(expected = TopologyException.class) + public void testAddSinkWithWrongParent() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addSink("sink", "topic-2", "source"); + } + + @Test(expected = TopologyException.class) + public void testAddSinkWithSelfParent() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addSink("sink", "topic-2", "sink"); + } + + @Test + public void testSourceTopics() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addSource("source-1", "topic-1"); + builder.addSource("source-2", "topic-2"); + builder.addSource("source-3", "topic-3"); + + assertEquals(builder.sourceTopics().size(), 3); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/MinTimestampTrackerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/MinTimestampTrackerTest.java new file mode 100644 index 0000000000000..0a1f95c29e0a8 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/MinTimestampTrackerTest.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.streams.processor.internals; + +import static org.junit.Assert.assertEquals; + +import org.junit.Test; + +public class MinTimestampTrackerTest { + + private Stamped elem(long timestamp) { + return new Stamped<>("", timestamp); + } + + @SuppressWarnings("unchecked") + @Test + public void testTracking() { + TimestampTracker tracker = new MinTimestampTracker<>(); + + Object[] elems = new Object[]{ + elem(100), elem(101), elem(102), elem(98), elem(99), elem(100) + }; + + int insertionIndex = 0; + int removalIndex = 0; + + // add 100 + tracker.addElement((Stamped) elems[insertionIndex++]); + assertEquals(100L, tracker.get()); + + // add 101 + tracker.addElement((Stamped) elems[insertionIndex++]); + assertEquals(100L, tracker.get()); + + // remove 100 + tracker.removeElement((Stamped) elems[removalIndex++]); + assertEquals(101L, tracker.get()); + + // add 102 + tracker.addElement((Stamped) elems[insertionIndex++]); + assertEquals(101L, tracker.get()); + + // add 98 + tracker.addElement((Stamped) elems[insertionIndex++]); + assertEquals(98L, tracker.get()); + + // add 99 + tracker.addElement((Stamped) elems[insertionIndex++]); + assertEquals(98L, tracker.get()); + + // add 100 + tracker.addElement((Stamped) elems[insertionIndex++]); + assertEquals(98L, tracker.get()); + + // remove 101 + tracker.removeElement((Stamped) elems[removalIndex++]); + assertEquals(98L, tracker.get()); + + // remove 102 + tracker.removeElement((Stamped) elems[removalIndex++]); + assertEquals(98L, tracker.get()); + + // remove 98 + tracker.removeElement((Stamped) elems[removalIndex++]); + assertEquals(99L, tracker.get()); + + // remove 99 + tracker.removeElement((Stamped) elems[removalIndex++]); + assertEquals(100L, tracker.get()); + + // remove 100 + tracker.removeElement((Stamped) elems[removalIndex++]); + assertEquals(100L, tracker.get()); + + assertEquals(insertionIndex, removalIndex); + } + +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java new file mode 100644 index 0000000000000..b91acdce13bfc --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java @@ -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 org.apache.kafka.streams.processor.internals; + +import static org.junit.Assert.assertEquals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.processor.TimestampExtractor; +import org.apache.kafka.test.MockSourceNode; +import org.apache.kafka.test.MockTimestampExtractor; +import org.junit.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; + +public class PartitionGroupTest { + private final Serializer intSerializer = new IntegerSerializer(); + private final Deserializer intDeserializer = new IntegerDeserializer(); + private final TimestampExtractor timestampExtractor = new MockTimestampExtractor(); + private final TopicPartition partition1 = new TopicPartition("topic", 1); + private final TopicPartition partition2 = new TopicPartition("topic", 2); + private final RecordQueue queue1 = new RecordQueue(partition1, new MockSourceNode<>(intDeserializer, intDeserializer)); + private final RecordQueue queue2 = new RecordQueue(partition2, new MockSourceNode<>(intDeserializer, intDeserializer)); + + private final byte[] recordValue = intSerializer.serialize(null, 10); + private final byte[] recordKey = intSerializer.serialize(null, 1); + + private final PartitionGroup group = new PartitionGroup(new HashMap() { + { + put(partition1, queue1); + put(partition2, queue2); + } + }, timestampExtractor); + + @Test + public void testTimeTracking() { + assertEquals(0, group.numBuffered()); + + // add three 3 records with timestamp 1, 3, 5 to partition-1 + List> list1 = Arrays.asList( + new ConsumerRecord<>("topic", 1, 1, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 3, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 5, recordKey, recordValue)); + + group.addRawRecords(partition1, list1); + + // add three 3 records with timestamp 2, 4, 6 to partition-2 + List> list2 = Arrays.asList( + new ConsumerRecord<>("topic", 1, 2, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 4, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 6, recordKey, recordValue)); + + group.addRawRecords(partition2, list2); + + assertEquals(6, group.numBuffered()); + assertEquals(3, group.numBuffered(partition1)); + assertEquals(3, group.numBuffered(partition2)); + assertEquals(TimestampTracker.NOT_KNOWN, group.timestamp()); + + StampedRecord record; + PartitionGroup.RecordInfo info = new PartitionGroup.RecordInfo(); + + // get one record + record = group.nextRecord(info); + assertEquals(partition1, info.partition()); + assertEquals(1L, record.timestamp); + assertEquals(5, group.numBuffered()); + assertEquals(2, group.numBuffered(partition1)); + assertEquals(3, group.numBuffered(partition2)); + assertEquals(TimestampTracker.NOT_KNOWN, group.timestamp()); + + // get one record, now the time should be advanced + record = group.nextRecord(info); + assertEquals(partition2, info.partition()); + assertEquals(2L, record.timestamp); + assertEquals(4, group.numBuffered()); + assertEquals(2, group.numBuffered(partition1)); + assertEquals(2, group.numBuffered(partition2)); + assertEquals(3L, group.timestamp()); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java new file mode 100644 index 0000000000000..eb33dc3596044 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java @@ -0,0 +1,449 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams.processor.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.MockConsumer; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.processor.StateRestoreCallback; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.state.OffsetCheckpoint; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.nio.channels.FileLock; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertFalse; + +public class ProcessorStateManagerTest { + + private static class MockStateStore implements StateStore { + private final String name; + private final boolean persistent; + + public boolean flushed = false; + public boolean closed = false; + public final ArrayList keys = new ArrayList<>(); + + public MockStateStore(String name, boolean persistent) { + this.name = name; + this.persistent = persistent; + } + @Override + public String name() { + return name; + } + @Override + public void flush() { + flushed = true; + } + @Override + public void close() { + closed = true; + } + @Override + public boolean persistent() { + return persistent; + } + + public final StateRestoreCallback stateRestoreCallback = new StateRestoreCallback() { + private final Deserializer deserializer = new IntegerDeserializer(); + + @Override + public void restore(byte[] key, byte[] value) { + keys.add(deserializer.deserialize("", key)); + } + }; + } + + private class MockRestoreConsumer extends MockConsumer { + private final Serializer serializer = new IntegerSerializer(); + + public TopicPartition assignedPartition = null; + public TopicPartition seekPartition = null; + public long seekOffset = -1L; + public boolean seekToBeginingCalled = false; + public boolean seekToEndCalled = false; + private long endOffset = 0L; + private long currentOffset = 0L; + + private ArrayList> recordBuffer = new ArrayList<>(); + + MockRestoreConsumer() { + super(OffsetResetStrategy.EARLIEST); + + reset(); + } + + // reset this mock restore consumer for a state store registration + public void reset() { + assignedPartition = null; + seekOffset = -1L; + seekToBeginingCalled = false; + seekToEndCalled = false; + endOffset = 0L; + recordBuffer.clear(); + } + + // buffer a record (we cannot use addRecord because we need to add records before asigning a partition) + public void bufferRecord(ConsumerRecord record) { + recordBuffer.add( + new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), + serializer.serialize(record.topic(), record.key()), + serializer.serialize(record.topic(), record.value()))); + endOffset = record.offset(); + + super.updateEndOffsets(Collections.singletonMap(assignedPartition, endOffset)); + } + + @Override + public synchronized void assign(List partitions) { + int numPartitions = partitions.size(); + if (numPartitions > 1) + throw new IllegalArgumentException("RestoreConsumer: more than one partition specified"); + + if (numPartitions == 1) { + if (assignedPartition != null) + throw new IllegalStateException("RestoreConsumer: partition already assigned"); + assignedPartition = partitions.get(0); + + // set the beginning offset to 0 + // NOTE: this is users responsible to set the initial lEO. + super.updateBeginningOffsets(Collections.singletonMap(assignedPartition, 0L)); + } + + super.assign(partitions); + } + + @Override + public ConsumerRecords poll(long timeout) { + // add buffered records to MockConsumer + for (ConsumerRecord record : recordBuffer) { + super.addRecord(record); + } + recordBuffer.clear(); + + ConsumerRecords records = super.poll(timeout); + + // set the current offset + Iterable> partitionRecords = records.records(assignedPartition); + for (ConsumerRecord record : partitionRecords) { + currentOffset = record.offset(); + } + + return records; + } + + @Override + public synchronized long position(TopicPartition partition) { + if (!partition.equals(assignedPartition)) + throw new IllegalStateException("RestoreConsumer: unassigned partition"); + + return currentOffset; + } + + @Override + public synchronized void seek(TopicPartition partition, long offset) { + if (offset < 0) + throw new IllegalArgumentException("RestoreConsumer: offset should not be negative"); + + if (seekOffset >= 0) + throw new IllegalStateException("RestoreConsumer: offset already seeked"); + + seekPartition = partition; + seekOffset = offset; + currentOffset = offset; + super.seek(partition, offset); + } + + @Override + public synchronized void seekToBeginning(TopicPartition... partitions) { + if (partitions.length != 1) + throw new IllegalStateException("RestoreConsumer: other than one partition specified"); + + for (TopicPartition partition : partitions) { + if (!partition.equals(assignedPartition)) + throw new IllegalStateException("RestoreConsumer: seek-to-end not on the assigned partition"); + } + + seekToBeginingCalled = true; + currentOffset = 0L; + } + + @Override + public synchronized void seekToEnd(TopicPartition... partitions) { + if (partitions.length != 1) + throw new IllegalStateException("RestoreConsumer: other than one partition specified"); + + for (TopicPartition partition : partitions) { + if (!partition.equals(assignedPartition)) + throw new IllegalStateException("RestoreConsumer: seek-to-end not on the assigned partition"); + } + + seekToEndCalled = true; + currentOffset = endOffset; + } + } + + @Test + public void testLockStateDirectory() throws IOException { + File baseDir = Files.createTempDirectory("test").toFile(); + try { + FileLock lock; + + // the state manager locks the directory + ProcessorStateManager stateMgr = new ProcessorStateManager(1, baseDir, new MockRestoreConsumer()); + + try { + // this should not get the lock + lock = ProcessorStateManager.lockStateDirectory(baseDir); + assertNull(lock); + } finally { + // by closing the state manager, release the lock + stateMgr.close(Collections.emptyMap()); + } + + // now, this should get the lock + lock = ProcessorStateManager.lockStateDirectory(baseDir); + try { + assertNotNull(lock); + } finally { + if (lock != null) lock.release(); + } + } finally { + Utils.delete(baseDir); + } + } + + @Test(expected = IllegalStateException.class) + public void testNoTopic() throws IOException { + File baseDir = Files.createTempDirectory("test").toFile(); + try { + MockStateStore mockStateStore = new MockStateStore("mockStore", false); + + ProcessorStateManager stateMgr = new ProcessorStateManager(1, baseDir, new MockRestoreConsumer()); + try { + stateMgr.register(mockStateStore, mockStateStore.stateRestoreCallback); + } finally { + stateMgr.close(Collections.emptyMap()); + } + } finally { + Utils.delete(baseDir); + } + } + + @Test + public void testRegisterPersistentStore() throws IOException { + File baseDir = Files.createTempDirectory("test").toFile(); + try { + long lastCheckpointedOffset = 10L; + OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(baseDir, ProcessorStateManager.CHECKPOINT_FILE_NAME)); + checkpoint.write(Collections.singletonMap(new TopicPartition("persistentStore", 2), lastCheckpointedOffset)); + + MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); + restoreConsumer.updatePartitions("persistentStore", Arrays.asList( + new PartitionInfo("persistentStore", 1, Node.noNode(), new Node[0], new Node[0]), + new PartitionInfo("persistentStore", 2, Node.noNode(), new Node[0], new Node[0]) + )); + restoreConsumer.updateEndOffsets(Collections.singletonMap(new TopicPartition("persistentStore", 2), 13L)); + + MockStateStore persistentStore = new MockStateStore("persistentStore", true); // persistent store + + ProcessorStateManager stateMgr = new ProcessorStateManager(2, baseDir, restoreConsumer); + try { + restoreConsumer.reset(); + + ArrayList expectedKeys = new ArrayList<>(); + for (int i = 1; i <= 3; i++) { + long offset = (long) i; + int key = i * 10; + expectedKeys.add(key); + restoreConsumer.bufferRecord( + new ConsumerRecord<>("persistentStore", 2, offset, key, 0) + ); + } + + stateMgr.register(persistentStore, persistentStore.stateRestoreCallback); + + assertEquals(new TopicPartition("persistentStore", 2), restoreConsumer.assignedPartition); + assertEquals(lastCheckpointedOffset, restoreConsumer.seekOffset); + assertFalse(restoreConsumer.seekToBeginingCalled); + assertTrue(restoreConsumer.seekToEndCalled); + assertEquals(expectedKeys, persistentStore.keys); + + } finally { + stateMgr.close(Collections.emptyMap()); + } + + } finally { + Utils.delete(baseDir); + } + } + + @Test + public void testRegisterNonPersistentStore() throws IOException { + File baseDir = Files.createTempDirectory("test").toFile(); + try { + long lastCheckpointedOffset = 10L; + OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(baseDir, ProcessorStateManager.CHECKPOINT_FILE_NAME)); + checkpoint.write(Collections.singletonMap(new TopicPartition("persistentStore", 2), lastCheckpointedOffset)); + + MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); + restoreConsumer.updatePartitions("nonPersistentStore", Arrays.asList( + new PartitionInfo("nonPersistentStore", 1, Node.noNode(), new Node[0], new Node[0]), + new PartitionInfo("nonPersistentStore", 2, Node.noNode(), new Node[0], new Node[0]) + )); + restoreConsumer.updateEndOffsets(Collections.singletonMap(new TopicPartition("persistentStore", 2), 13L)); + + MockStateStore nonPersistentStore = new MockStateStore("nonPersistentStore", false); // non persistent store + + ProcessorStateManager stateMgr = new ProcessorStateManager(2, baseDir, restoreConsumer); + try { + restoreConsumer.reset(); + + ArrayList expectedKeys = new ArrayList<>(); + for (int i = 1; i <= 3; i++) { + long offset = (long) (i + 100); + int key = i; + expectedKeys.add(i); + restoreConsumer.bufferRecord( + new ConsumerRecord<>("nonPersistentStore", 2, offset, key, 0) + ); + } + + stateMgr.register(nonPersistentStore, nonPersistentStore.stateRestoreCallback); + + assertEquals(new TopicPartition("nonPersistentStore", 2), restoreConsumer.assignedPartition); + assertEquals(0L, restoreConsumer.seekOffset); + assertTrue(restoreConsumer.seekToBeginingCalled); + assertTrue(restoreConsumer.seekToEndCalled); + assertEquals(expectedKeys, nonPersistentStore.keys); + } finally { + stateMgr.close(Collections.emptyMap()); + } + } finally { + Utils.delete(baseDir); + } + } + + @Test + public void testGetStore() throws IOException { + File baseDir = Files.createTempDirectory("test").toFile(); + try { + MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); + restoreConsumer.updatePartitions("mockStore", Arrays.asList( + new PartitionInfo("mockStore", 1, Node.noNode(), new Node[0], new Node[0]) + )); + + MockStateStore mockStateStore = new MockStateStore("mockStore", false); + + ProcessorStateManager stateMgr = new ProcessorStateManager(1, baseDir, restoreConsumer); + try { + stateMgr.register(mockStateStore, mockStateStore.stateRestoreCallback); + + assertNull(stateMgr.getStore("noSuchStore")); + assertEquals(mockStateStore, stateMgr.getStore("mockStore")); + + } finally { + stateMgr.close(Collections.emptyMap()); + } + } finally { + Utils.delete(baseDir); + } + } + + @Test + public void testClose() throws IOException { + File baseDir = Files.createTempDirectory("test").toFile(); + File checkpointFile = new File(baseDir, ProcessorStateManager.CHECKPOINT_FILE_NAME); + try { + // write an empty checkpoint file + OffsetCheckpoint oldCheckpoint = new OffsetCheckpoint(checkpointFile); + oldCheckpoint.write(Collections.emptyMap()); + + MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); + restoreConsumer.updatePartitions("persistentStore", Arrays.asList( + new PartitionInfo("persistentStore", 1, Node.noNode(), new Node[0], new Node[0]) + )); + restoreConsumer.updatePartitions("nonPersistentStore", Arrays.asList( + new PartitionInfo("nonPersistentStore", 1, Node.noNode(), new Node[0], new Node[0]) + )); + + // set up ack'ed offsets + HashMap ackedOffsets = new HashMap<>(); + ackedOffsets.put(new TopicPartition("persistentStore", 1), 123L); + ackedOffsets.put(new TopicPartition("nonPersistentStore", 1), 456L); + ackedOffsets.put(new TopicPartition("otherTopic", 1), 789L); + + MockStateStore persistentStore = new MockStateStore("persistentStore", true); + MockStateStore nonPersistentStore = new MockStateStore("nonPersistentStore", false); + + ProcessorStateManager stateMgr = new ProcessorStateManager(1, baseDir, restoreConsumer); + try { + // make sure the checkpoint file is deleted + assertFalse(checkpointFile.exists()); + + restoreConsumer.reset(); + stateMgr.register(persistentStore, persistentStore.stateRestoreCallback); + + restoreConsumer.reset(); + stateMgr.register(nonPersistentStore, nonPersistentStore.stateRestoreCallback); + } finally { + // close the state manager with the ack'ed offsets + stateMgr.close(ackedOffsets); + } + + // make sure all stores are closed, and the checkpoint file is written. + assertTrue(persistentStore.flushed); + assertTrue(persistentStore.closed); + assertTrue(nonPersistentStore.flushed); + assertTrue(nonPersistentStore.closed); + assertTrue(checkpointFile.exists()); + + // the checkpoint file should contain an offset from the persistent store only. + OffsetCheckpoint newCheckpoint = new OffsetCheckpoint(checkpointFile); + Map checkpointedOffsets = newCheckpoint.read(); + assertEquals(1, checkpointedOffsets.size()); + assertEquals(new Long(123L + 1L), checkpointedOffsets.get(new TopicPartition("persistentStore", 1))); + } finally { + Utils.delete(baseDir); + } + } + +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java new file mode 100644 index 0000000000000..50a23ecbbe750 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java @@ -0,0 +1,326 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams.processor.internals; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.streams.StreamingConfig; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.TimestampExtractor; +import org.apache.kafka.streams.processor.TopologyBuilder; +import org.apache.kafka.streams.state.InMemoryKeyValueStore; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.ProcessorTopologyTestDriver; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.nio.file.FileVisitResult; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.SimpleFileVisitor; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.Properties; + +public class ProcessorTopologyTest { + + private static final Serializer STRING_SERIALIZER = new StringSerializer(); + private static final Deserializer STRING_DESERIALIZER = new StringDeserializer(); + private static final File STATE_DIR = new File("build/data").getAbsoluteFile(); + + protected static final String INPUT_TOPIC = "input-topic"; + protected static final String OUTPUT_TOPIC_1 = "output-topic-1"; + protected static final String OUTPUT_TOPIC_2 = "output-topic-2"; + + private static long timestamp = 1000L; + + private ProcessorTopologyTestDriver driver; + private StreamingConfig config; + + @Before + public void setup() { + STATE_DIR.mkdirs(); + Properties props = new Properties(); + props.setProperty(StreamingConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9091"); + props.setProperty(StreamingConfig.STATE_DIR_CONFIG, STATE_DIR.getAbsolutePath()); + props.setProperty(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, CustomTimestampExtractor.class.getName()); + props.setProperty(StreamingConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); + props.setProperty(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); + props.setProperty(StreamingConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); + props.setProperty(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); + this.config = new StreamingConfig(props); + } + + @After + public void cleanup() { + if (driver != null) { + driver.close(); + } + driver = null; + if (STATE_DIR.exists()) { + try { + Files.walkFileTree(STATE_DIR.toPath(), new SimpleFileVisitor() { + @Override + public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException { + Files.delete(file); + return FileVisitResult.CONTINUE; + } + + @Override + public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOException { + Files.delete(dir); + return FileVisitResult.CONTINUE; + } + + }); + } catch (IOException e) { + // do nothing + } + } + } + + @Test + public void testTopologyMetadata() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addSource("source-1", "topic-1"); + builder.addSource("source-2", "topic-2", "topic-3"); + builder.addProcessor("processor-1", new MockProcessorSupplier(), "source-1"); + builder.addProcessor("processor-2", new MockProcessorSupplier(), "source-1", "source-2"); + builder.addSink("sink-1", "topic-3", "processor-1"); + builder.addSink("sink-2", "topic-4", "processor-1", "processor-2"); + + final ProcessorTopology topology = builder.build(); + + assertEquals(6, topology.processors().size()); + + assertEquals(2, topology.sources().size()); + + assertEquals(3, topology.sourceTopics().size()); + + assertNotNull(topology.source("topic-1")); + + assertNotNull(topology.source("topic-2")); + + assertNotNull(topology.source("topic-3")); + + assertEquals(topology.source("topic-2"), topology.source("topic-3")); + } + + @Test + public void testDrivingSimpleTopology() { + driver = new ProcessorTopologyTestDriver(config, createSimpleTopology()); + driver.process(INPUT_TOPIC, "key1", "value1", STRING_SERIALIZER, STRING_SERIALIZER); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key1", "value1"); + assertNoOutputRecord(OUTPUT_TOPIC_2); + + driver.process(INPUT_TOPIC, "key2", "value2", STRING_SERIALIZER, STRING_SERIALIZER); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key2", "value2"); + assertNoOutputRecord(OUTPUT_TOPIC_2); + + driver.process(INPUT_TOPIC, "key3", "value3", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key4", "value4", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key5", "value5", STRING_SERIALIZER, STRING_SERIALIZER); + assertNoOutputRecord(OUTPUT_TOPIC_2); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key3", "value3"); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key4", "value4"); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key5", "value5"); + } + + @Test + public void testDrivingMultiplexingTopology() { + driver = new ProcessorTopologyTestDriver(config, createMultiplexingTopology()); + driver.process(INPUT_TOPIC, "key1", "value1", STRING_SERIALIZER, STRING_SERIALIZER); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key1", "value1(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key1", "value1(2)"); + + driver.process(INPUT_TOPIC, "key2", "value2", STRING_SERIALIZER, STRING_SERIALIZER); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key2", "value2(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key2", "value2(2)"); + + driver.process(INPUT_TOPIC, "key3", "value3", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key4", "value4", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key5", "value5", STRING_SERIALIZER, STRING_SERIALIZER); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key3", "value3(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key4", "value4(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key5", "value5(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key3", "value3(2)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key4", "value4(2)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key5", "value5(2)"); + } + + @Test + public void testDrivingStatefulTopology() { + String storeName = "entries"; + driver = new ProcessorTopologyTestDriver(config, createStatefulTopology(storeName), storeName); + driver.process(INPUT_TOPIC, "key1", "value1", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key2", "value2", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key3", "value3", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key1", "value4", STRING_SERIALIZER, STRING_SERIALIZER); + assertNoOutputRecord(OUTPUT_TOPIC_1); + + KeyValueStore store = driver.getKeyValueStore("entries"); + assertEquals("value4", store.get("key1")); + assertEquals("value2", store.get("key2")); + assertEquals("value3", store.get("key3")); + assertNull(store.get("key4")); + } + + protected void assertNextOutputRecord(String topic, String key, String value) { + assertProducerRecord(driver.readOutput(topic, STRING_DESERIALIZER, STRING_DESERIALIZER), topic, key, value); + } + + protected void assertNoOutputRecord(String topic) { + assertNull(driver.readOutput(topic)); + } + + private void assertProducerRecord(ProducerRecord record, String topic, String key, String value) { + assertEquals(topic, record.topic()); + assertEquals(key, record.key()); + assertEquals(value, record.value()); + // Kafka Streaming doesn't set the partition, so it's always null + assertNull(record.partition()); + } + + protected TopologyBuilder createSimpleTopology() { + return new TopologyBuilder().addSource("source", STRING_DESERIALIZER, STRING_DESERIALIZER, INPUT_TOPIC) + .addProcessor("processor", define(new ForwardingProcessor()), "source") + .addSink("sink", OUTPUT_TOPIC_1, "processor"); + } + + protected TopologyBuilder createMultiplexingTopology() { + return new TopologyBuilder().addSource("source", STRING_DESERIALIZER, STRING_DESERIALIZER, INPUT_TOPIC) + .addProcessor("processor", define(new MultiplexingProcessor(2)), "source") + .addSink("sink1", OUTPUT_TOPIC_1, "processor") + .addSink("sink2", OUTPUT_TOPIC_2, "processor"); + } + + protected TopologyBuilder createStatefulTopology(String storeName) { + return new TopologyBuilder().addSource("source", STRING_DESERIALIZER, STRING_DESERIALIZER, INPUT_TOPIC) + .addProcessor("processor", define(new StatefulProcessor(storeName)), "source") + .addSink("counts", OUTPUT_TOPIC_1, "processor"); + } + + /** + * A processor that simply forwards all messages to all children. + */ + protected static class ForwardingProcessor extends AbstractProcessor { + + @Override + public void process(String key, String value) { + context().forward(key, value); + } + + @Override + public void punctuate(long streamTime) { + context().forward(Long.toString(streamTime), "punctuate"); + } + } + + /** + * A processor that forwards slightly-modified messages to each child. + */ + protected static class MultiplexingProcessor extends AbstractProcessor { + + private final int numChildren; + + public MultiplexingProcessor(int numChildren) { + this.numChildren = numChildren; + } + + @Override + public void process(String key, String value) { + for (int i = 0; i != numChildren; ++i) { + context().forward(key, value + "(" + (i + 1) + ")", i); + } + } + + @Override + public void punctuate(long streamTime) { + for (int i = 0; i != numChildren; ++i) { + context().forward(Long.toString(streamTime), "punctuate(" + (i + 1) + ")", i); + } + } + } + + /** + * A processor that stores each key-value pair in an in-memory key-value store registered with the context. When + * {@link #punctuate(long)} is called, it outputs the total number of entries in the store. + */ + protected static class StatefulProcessor extends AbstractProcessor { + + private KeyValueStore store; + private final String storeName; + + public StatefulProcessor(String storeName) { + this.storeName = storeName; + } + + @Override + public void init(ProcessorContext context) { + super.init(context); + store = new InMemoryKeyValueStore<>(storeName, context); + } + + @Override + public void process(String key, String value) { + store.put(key, value); + } + + @Override + public void punctuate(long streamTime) { + int count = 0; + for (KeyValueIterator iter = store.all(); iter.hasNext();) { + iter.next(); + ++count; + } + context().forward(Long.toString(streamTime), count); + } + } + + protected ProcessorSupplier define(final Processor processor) { + return new ProcessorSupplier() { + @Override + public Processor get() { + return processor; + } + }; + } + + public static class CustomTimestampExtractor implements TimestampExtractor { + @Override + public long extract(ConsumerRecord record) { + return timestamp; + } + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PunctuationQueueTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PunctuationQueueTest.java new file mode 100644 index 0000000000000..b1403bd02f44c --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PunctuationQueueTest.java @@ -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 org.apache.kafka.streams.processor.internals; + +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.junit.Test; + +import java.util.ArrayList; + +import static org.junit.Assert.assertEquals; + +public class PunctuationQueueTest { + + @Test + public void testPunctuationInterval() { + TestProcessor processor = new TestProcessor(); + ProcessorNode node = new ProcessorNode<>("test", processor); + PunctuationQueue queue = new PunctuationQueue(); + + PunctuationSchedule sched = new PunctuationSchedule(node, 100L); + final long now = sched.timestamp - 100L; + + queue.schedule(sched); + + Punctuator punctuator = new Punctuator() { + public void punctuate(ProcessorNode node, long time) { + node.processor().punctuate(time); + } + }; + + queue.mayPunctuate(now, punctuator); + assertEquals(0, processor.punctuatedAt.size()); + + queue.mayPunctuate(now + 99L, punctuator); + assertEquals(0, processor.punctuatedAt.size()); + + queue.mayPunctuate(now + 100L, punctuator); + assertEquals(1, processor.punctuatedAt.size()); + + queue.mayPunctuate(now + 199L, punctuator); + assertEquals(1, processor.punctuatedAt.size()); + + queue.mayPunctuate(now + 200L, punctuator); + assertEquals(2, processor.punctuatedAt.size()); + } + + private static class TestProcessor implements Processor { + + public final ArrayList punctuatedAt = new ArrayList<>(); + + @Override + public void init(ProcessorContext context) { + } + + @Override + public void process(String key, String value) { + } + + @Override + public void punctuate(long streamTime) { + punctuatedAt.add(streamTime); + } + + @Override + public void close() { + } + } + +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java new file mode 100644 index 0000000000000..6e86410b19ed6 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java @@ -0,0 +1,116 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

                  + * http://www.apache.org/licenses/LICENSE-2.0 + *

                  + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams.processor.internals; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.processor.TimestampExtractor; +import org.apache.kafka.test.MockSourceNode; +import org.apache.kafka.test.MockTimestampExtractor; +import org.junit.Test; + +import java.util.Arrays; +import java.util.List; + +public class RecordQueueTest { + private final Serializer intSerializer = new IntegerSerializer(); + private final Deserializer intDeserializer = new IntegerDeserializer(); + private final TimestampExtractor timestampExtractor = new MockTimestampExtractor(); + private final RecordQueue queue = new RecordQueue(new TopicPartition("topic", 1), new MockSourceNode<>(intDeserializer, intDeserializer)); + + private final byte[] recordValue = intSerializer.serialize(null, 10); + private final byte[] recordKey = intSerializer.serialize(null, 1); + + @Test + public void testTimeTracking() { + + assertTrue(queue.isEmpty()); + + // add three 3 out-of-order records with timestamp 2, 1, 3 + List> list1 = Arrays.asList( + new ConsumerRecord<>("topic", 1, 2, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 1, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 3, recordKey, recordValue)); + + queue.addRawRecords(list1, timestampExtractor); + + assertEquals(3, queue.size()); + assertEquals(TimestampTracker.NOT_KNOWN, queue.timestamp()); + + // poll the first record, now with 1, 3 + assertEquals(2L, queue.poll().timestamp); + assertEquals(2, queue.size()); + assertEquals(1L, queue.timestamp()); + + // poll the second record, now with 3 + assertEquals(1L, queue.poll().timestamp); + assertEquals(1, queue.size()); + assertEquals(3L, queue.timestamp()); + + // add three 3 out-of-order records with timestamp 4, 1, 2 + // now with 3, 4, 1, 2 + List> list2 = Arrays.asList( + new ConsumerRecord<>("topic", 1, 4, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 1, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 2, recordKey, recordValue)); + + queue.addRawRecords(list2, timestampExtractor); + + assertEquals(4, queue.size()); + assertEquals(3L, queue.timestamp()); + + // poll the third record, now with 4, 1, 2 + assertEquals(3L, queue.poll().timestamp); + assertEquals(3, queue.size()); + assertEquals(3L, queue.timestamp()); + + // poll the rest records + assertEquals(4L, queue.poll().timestamp); + assertEquals(3L, queue.timestamp()); + + assertEquals(1L, queue.poll().timestamp); + assertEquals(3L, queue.timestamp()); + + assertEquals(2L, queue.poll().timestamp); + assertEquals(0, queue.size()); + assertEquals(3L, queue.timestamp()); + + // add three more records with 4, 5, 6 + List> list3 = Arrays.asList( + new ConsumerRecord<>("topic", 1, 4, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 5, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 6, recordKey, recordValue)); + + queue.addRawRecords(list3, timestampExtractor); + + assertEquals(3, queue.size()); + assertEquals(3L, queue.timestamp()); + + // poll one record again, the timestamp should advance now + assertEquals(4L, queue.poll().timestamp); + assertEquals(2, queue.size()); + assertEquals(5L, queue.timestamp()); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java new file mode 100644 index 0000000000000..f93093cf6fde0 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.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.streams.processor.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.MockConsumer; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.clients.producer.MockProducer; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.StreamingConfig; +import org.apache.kafka.test.MockSourceNode; +import org.junit.Test; +import org.junit.Before; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Properties; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class StreamTaskTest { + + private final Serializer intSerializer = new IntegerSerializer(); + private final Deserializer intDeserializer = new IntegerDeserializer(); + private final Serializer bytesSerializer = new ByteArraySerializer(); + + private final TopicPartition partition1 = new TopicPartition("topic1", 1); + private final TopicPartition partition2 = new TopicPartition("topic2", 1); + private final HashSet partitions = new HashSet<>(Arrays.asList(partition1, partition2)); + + private final MockSourceNode source1 = new MockSourceNode<>(intDeserializer, intDeserializer); + private final MockSourceNode source2 = new MockSourceNode<>(intDeserializer, intDeserializer); + private final ProcessorTopology topology = new ProcessorTopology( + Arrays.asList((ProcessorNode) source1, (ProcessorNode) source2), + new HashMap() { + { + put("topic1", source1); + put("topic2", source2); + } + }); + + private final StreamingConfig config = new StreamingConfig(new Properties() { + { + setProperty(StreamingConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + setProperty(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + setProperty(StreamingConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + setProperty(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + setProperty(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, "org.apache.kafka.test.MockTimestampExtractor"); + setProperty(StreamingConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"); + setProperty(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"); + } + }); + + private final MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + private final MockProducer producer = new MockProducer<>(false, bytesSerializer, bytesSerializer); + private final MockConsumer restoreStateConsumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + + private final byte[] recordValue = intSerializer.serialize(null, 10); + private final byte[] recordKey = intSerializer.serialize(null, 1); + + + @Before + public void setup() { + consumer.assign(Arrays.asList(partition1, partition2)); + } + + @SuppressWarnings("unchecked") + @Test + public void testProcessOrder() { + StreamTask task = new StreamTask(0, consumer, producer, restoreStateConsumer, partitions, topology, config, null); + + task.addRecords(partition1, records( + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, recordKey, recordValue) + )); + + task.addRecords(partition2, records( + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 25, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, recordKey, recordValue) + )); + + assertEquals(task.process(), 5); + assertEquals(source1.numReceived, 1); + assertEquals(source2.numReceived, 0); + + assertEquals(task.process(), 4); + assertEquals(source1.numReceived, 1); + assertEquals(source2.numReceived, 1); + + assertEquals(task.process(), 3); + assertEquals(source1.numReceived, 2); + assertEquals(source2.numReceived, 1); + + assertEquals(task.process(), 2); + assertEquals(source1.numReceived, 3); + assertEquals(source2.numReceived, 1); + + assertEquals(task.process(), 1); + assertEquals(source1.numReceived, 3); + assertEquals(source2.numReceived, 2); + + assertEquals(task.process(), 0); + assertEquals(source1.numReceived, 3); + assertEquals(source2.numReceived, 3); + + task.close(); + } + + @SuppressWarnings("unchecked") + @Test + public void testPauseResume() { + StreamTask task = new StreamTask(1, consumer, producer, restoreStateConsumer, partitions, topology, config, null); + + task.addRecords(partition1, records( + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, recordKey, recordValue) + )); + + task.addRecords(partition2, records( + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 55, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 65, recordKey, recordValue) + )); + + assertEquals(task.process(), 5); + assertEquals(source1.numReceived, 1); + assertEquals(source2.numReceived, 0); + + assertEquals(consumer.paused().size(), 1); + assertTrue(consumer.paused().contains(partition2)); + + task.addRecords(partition1, records( + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 40, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 50, recordKey, recordValue) + )); + + assertEquals(consumer.paused().size(), 2); + assertTrue(consumer.paused().contains(partition1)); + assertTrue(consumer.paused().contains(partition2)); + + assertEquals(task.process(), 7); + assertEquals(source1.numReceived, 1); + assertEquals(source2.numReceived, 1); + + assertEquals(consumer.paused().size(), 1); + assertTrue(consumer.paused().contains(partition1)); + + assertEquals(task.process(), 6); + assertEquals(source1.numReceived, 2); + assertEquals(source2.numReceived, 1); + + assertEquals(consumer.paused().size(), 0); + + task.close(); + } + + private Iterable> records(ConsumerRecord... recs) { + return Arrays.asList(recs); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java new file mode 100644 index 0000000000000..a7e707e65de44 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -0,0 +1,390 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams.processor.internals; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.MockConsumer; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.clients.producer.MockProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.StreamingConfig; +import org.apache.kafka.streams.processor.TopologyBuilder; +import org.junit.Test; + +import java.io.File; +import java.nio.file.Files; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; + +public class StreamThreadTest { + + private TopicPartition t1p1 = new TopicPartition("topic1", 1); + private TopicPartition t1p2 = new TopicPartition("topic1", 2); + private TopicPartition t2p1 = new TopicPartition("topic2", 1); + private TopicPartition t2p2 = new TopicPartition("topic2", 2); + + private Properties configProps() { + return new Properties() { + { + setProperty(StreamingConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + setProperty(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + setProperty(StreamingConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + setProperty(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + setProperty(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, "org.apache.kafka.test.MockTimestampExtractor"); + setProperty(StreamingConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"); + setProperty(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"); + } + }; + } + + private static class TestStreamTask extends StreamTask { + public boolean committed = false; + + public TestStreamTask(int id, + Consumer consumer, + Producer producer, + Consumer restoreConsumer, + Collection partitions, + ProcessorTopology topology, + StreamingConfig config) { + super(id, consumer, producer, restoreConsumer, partitions, topology, config, null); + } + + @Override + public void commit() { + super.commit(); + committed = true; + } + } + + private ByteArraySerializer serializer = new ByteArraySerializer(); + + @SuppressWarnings("unchecked") + @Test + public void testPartitionAssignmentChange() throws Exception { + StreamingConfig config = new StreamingConfig(configProps()); + + MockProducer producer = new MockProducer<>(true, serializer, serializer); + MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + final MockConsumer mockRestoreConsumer = new MockConsumer<>(OffsetResetStrategy.LATEST); + + TopologyBuilder builder = new TopologyBuilder(); + builder.addSource("source1", "topic1"); + builder.addSource("source2", "topic2"); + + StreamThread thread = new StreamThread(builder, config, producer, consumer, mockRestoreConsumer, "test", new Metrics(), new SystemTime()) { + @Override + protected StreamTask createStreamTask(int id, Collection partitionsForTask) { + return new TestStreamTask(id, consumer, producer, mockRestoreConsumer, partitionsForTask, builder.build(), config); + } + }; + + ConsumerRebalanceListener rebalanceListener = thread.rebalanceListener; + + assertTrue(thread.tasks().isEmpty()); + + List revokedPartitions; + List assignedPartitions; + Set expectedGroup1; + Set expectedGroup2; + + revokedPartitions = Collections.emptyList(); + assignedPartitions = Collections.singletonList(t1p1); + expectedGroup1 = new HashSet<>(Arrays.asList(t1p1)); + + rebalanceListener.onPartitionsRevoked(revokedPartitions); + rebalanceListener.onPartitionsAssigned(assignedPartitions); + + assertTrue(thread.tasks().containsKey(1)); + assertEquals(expectedGroup1, thread.tasks().get(1).partitions()); + assertEquals(1, thread.tasks().size()); + + revokedPartitions = assignedPartitions; + assignedPartitions = Collections.singletonList(t1p2); + expectedGroup2 = new HashSet<>(Arrays.asList(t1p2)); + + rebalanceListener.onPartitionsRevoked(revokedPartitions); + rebalanceListener.onPartitionsAssigned(assignedPartitions); + + assertTrue(thread.tasks().containsKey(2)); + assertEquals(expectedGroup2, thread.tasks().get(2).partitions()); + assertEquals(1, thread.tasks().size()); + + revokedPartitions = assignedPartitions; + assignedPartitions = Arrays.asList(t1p1, t1p2); + expectedGroup1 = new HashSet<>(Collections.singleton(t1p1)); + expectedGroup2 = new HashSet<>(Collections.singleton(t1p2)); + + rebalanceListener.onPartitionsRevoked(revokedPartitions); + rebalanceListener.onPartitionsAssigned(assignedPartitions); + + assertTrue(thread.tasks().containsKey(1)); + assertTrue(thread.tasks().containsKey(2)); + assertEquals(expectedGroup1, thread.tasks().get(1).partitions()); + assertEquals(expectedGroup2, thread.tasks().get(2).partitions()); + assertEquals(2, thread.tasks().size()); + + revokedPartitions = assignedPartitions; + assignedPartitions = Arrays.asList(t1p1, t1p2, t2p1, t2p2); + expectedGroup1 = new HashSet<>(Arrays.asList(t1p1, t2p1)); + expectedGroup2 = new HashSet<>(Arrays.asList(t1p2, t2p2)); + + rebalanceListener.onPartitionsRevoked(revokedPartitions); + rebalanceListener.onPartitionsAssigned(assignedPartitions); + + assertTrue(thread.tasks().containsKey(1)); + assertTrue(thread.tasks().containsKey(2)); + assertEquals(expectedGroup1, thread.tasks().get(1).partitions()); + assertEquals(expectedGroup2, thread.tasks().get(2).partitions()); + assertEquals(2, thread.tasks().size()); + + revokedPartitions = assignedPartitions; + assignedPartitions = Collections.emptyList(); + + rebalanceListener.onPartitionsRevoked(revokedPartitions); + rebalanceListener.onPartitionsAssigned(assignedPartitions); + + assertTrue(thread.tasks().isEmpty()); + } + + @Test + public void testMaybeClean() throws Exception { + File baseDir = Files.createTempDirectory("test").toFile(); + try { + final long cleanupDelay = 1000L; + Properties props = configProps(); + props.setProperty(StreamingConfig.STATE_CLEANUP_DELAY_MS_CONFIG, Long.toString(cleanupDelay)); + props.setProperty(StreamingConfig.STATE_DIR_CONFIG, baseDir.getCanonicalPath()); + + StreamingConfig config = new StreamingConfig(props); + + File stateDir1 = new File(baseDir, "1"); + File stateDir2 = new File(baseDir, "2"); + File stateDir3 = new File(baseDir, "3"); + File extraDir = new File(baseDir, "X"); + stateDir1.mkdir(); + stateDir2.mkdir(); + stateDir3.mkdir(); + extraDir.mkdir(); + + MockProducer producer = new MockProducer<>(true, serializer, serializer); + MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + final MockConsumer mockRestoreConsumer = new MockConsumer<>(OffsetResetStrategy.LATEST); + MockTime mockTime = new MockTime(); + + TopologyBuilder builder = new TopologyBuilder(); + builder.addSource("source1", "topic1"); + + StreamThread thread = new StreamThread(builder, config, producer, consumer, mockRestoreConsumer, "test", new Metrics(), mockTime) { + @Override + public void maybeClean() { + super.maybeClean(); + } + @Override + protected StreamTask createStreamTask(int id, Collection partitionsForTask) { + return new TestStreamTask(id, consumer, producer, mockRestoreConsumer, partitionsForTask, builder.build(), config); + } + }; + + ConsumerRebalanceListener rebalanceListener = thread.rebalanceListener; + + assertTrue(thread.tasks().isEmpty()); + mockTime.sleep(cleanupDelay); + + // all directories exist since an assignment didn't happen + assertTrue(stateDir1.exists()); + assertTrue(stateDir2.exists()); + assertTrue(stateDir3.exists()); + assertTrue(extraDir.exists()); + + List revokedPartitions; + List assignedPartitions; + Map prevTasks; + + // + // Assign t1p1 and t1p2. This should create Task 1 & 2 + // + revokedPartitions = Collections.emptyList(); + assignedPartitions = Arrays.asList(t1p1, t1p2); + prevTasks = new HashMap(thread.tasks()); + + rebalanceListener.onPartitionsRevoked(revokedPartitions); + rebalanceListener.onPartitionsAssigned(assignedPartitions); + + // there shouldn't be any previous task + assertTrue(prevTasks.isEmpty()); + + // task 1 & 2 are created + assertEquals(2, thread.tasks().size()); + + // all directories should still exit before the cleanup delay time + mockTime.sleep(cleanupDelay - 10L); + thread.maybeClean(); + assertTrue(stateDir1.exists()); + assertTrue(stateDir2.exists()); + assertTrue(stateDir3.exists()); + assertTrue(extraDir.exists()); + + // all state directories except for task 1 & 2 will be removed. the extra directory should still exists + mockTime.sleep(11L); + thread.maybeClean(); + assertTrue(stateDir1.exists()); + assertTrue(stateDir2.exists()); + assertFalse(stateDir3.exists()); + assertTrue(extraDir.exists()); + + // + // Revoke t1p1 and t1p2. This should remove Task 1 & 2 + // + revokedPartitions = assignedPartitions; + assignedPartitions = Collections.emptyList(); + prevTasks = new HashMap(thread.tasks()); + + rebalanceListener.onPartitionsRevoked(revokedPartitions); + rebalanceListener.onPartitionsAssigned(assignedPartitions); + + // previous tasks should be committed + assertEquals(2, prevTasks.size()); + for (StreamTask task : prevTasks.values()) { + assertTrue(((TestStreamTask) task).committed); + ((TestStreamTask) task).committed = false; + } + + // no task + assertTrue(thread.tasks().isEmpty()); + + // all state directories for task 1 & 2 still exist before the cleanup delay time + mockTime.sleep(cleanupDelay - 10L); + thread.maybeClean(); + assertTrue(stateDir1.exists()); + assertTrue(stateDir2.exists()); + assertFalse(stateDir3.exists()); + assertTrue(extraDir.exists()); + + // all state directories for task 1 & 2 are removed + mockTime.sleep(11L); + thread.maybeClean(); + assertFalse(stateDir1.exists()); + assertFalse(stateDir2.exists()); + assertFalse(stateDir3.exists()); + assertTrue(extraDir.exists()); + + } finally { + Utils.delete(baseDir); + } + } + + @Test + public void testMaybeCommit() throws Exception { + File baseDir = Files.createTempDirectory("test").toFile(); + try { + final long commitInterval = 1000L; + Properties props = configProps(); + props.setProperty(StreamingConfig.STATE_DIR_CONFIG, baseDir.getCanonicalPath()); + props.setProperty(StreamingConfig.COMMIT_INTERVAL_MS_CONFIG, Long.toString(commitInterval)); + + StreamingConfig config = new StreamingConfig(props); + + MockProducer producer = new MockProducer<>(true, serializer, serializer); + MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + final MockConsumer mockRestoreConsumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + MockTime mockTime = new MockTime(); + + TopologyBuilder builder = new TopologyBuilder(); + builder.addSource("source1", "topic1"); + + StreamThread thread = new StreamThread(builder, config, producer, consumer, mockRestoreConsumer, "test", new Metrics(), mockTime) { + @Override + public void maybeCommit() { + super.maybeCommit(); + } + @Override + protected StreamTask createStreamTask(int id, Collection partitionsForTask) { + return new TestStreamTask(id, consumer, producer, mockRestoreConsumer, partitionsForTask, builder.build(), config); + } + }; + + ConsumerRebalanceListener rebalanceListener = thread.rebalanceListener; + + List revokedPartitions; + List assignedPartitions; + + // + // Assign t1p1 and t1p2. This should create Task 1 & 2 + // + revokedPartitions = Collections.emptyList(); + assignedPartitions = Arrays.asList(t1p1, t1p2); + + rebalanceListener.onPartitionsRevoked(revokedPartitions); + rebalanceListener.onPartitionsAssigned(assignedPartitions); + + assertEquals(2, thread.tasks().size()); + + // no task is committed before the commit interval + mockTime.sleep(commitInterval - 10L); + thread.maybeCommit(); + for (StreamTask task : thread.tasks().values()) { + assertFalse(((TestStreamTask) task).committed); + } + + // all tasks are committed after the commit interval + mockTime.sleep(11L); + thread.maybeCommit(); + for (StreamTask task : thread.tasks().values()) { + assertTrue(((TestStreamTask) task).committed); + ((TestStreamTask) task).committed = false; + } + + // no task is committed before the commit interval, again + mockTime.sleep(commitInterval - 10L); + thread.maybeCommit(); + for (StreamTask task : thread.tasks().values()) { + assertFalse(((TestStreamTask) task).committed); + } + + // all tasks are committed after the commit interval, again + mockTime.sleep(11L); + thread.maybeCommit(); + for (StreamTask task : thread.tasks().values()) { + assertTrue(((TestStreamTask) task).committed); + ((TestStreamTask) task).committed = false; + } + + } finally { + Utils.delete(baseDir); + } + } +} diff --git a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java new file mode 100644 index 0000000000000..2c42e6c225c90 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.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.test; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.internals.ProcessorNode; +import org.apache.kafka.streams.processor.internals.ProcessorTopology; + +import java.util.List; + +public class KStreamTestDriver { + + private final ProcessorTopology topology; + private final MockProcessorContext context; + private ProcessorNode currNode; + + public KStreamTestDriver(KStreamBuilder builder) { + this(builder, null, null); + } + + public KStreamTestDriver(KStreamBuilder builder, Serializer serializer, Deserializer deserializer) { + this.topology = builder.build(); + this.context = new MockProcessorContext(this, serializer, deserializer); + + for (ProcessorNode node : topology.processors()) { + currNode = node; + try { + node.init(context); + } finally { + currNode = null; + } + } + } + + public void process(String topicName, Object key, Object value) { + currNode = topology.source(topicName); + try { + forward(key, value); + } finally { + currNode = null; + } + } + + public void setTime(long timestamp) { + context.setTime(timestamp); + } + + public StateStore getStateStore(String name) { + return context.getStateStore(name); + } + + @SuppressWarnings("unchecked") + public void forward(K key, V value) { + ProcessorNode thisNode = currNode; + for (ProcessorNode childNode : (List>) thisNode.children()) { + currNode = childNode; + try { + childNode.process(key, value); + } finally { + currNode = thisNode; + } + } + } + + @SuppressWarnings("unchecked") + public void forward(K key, V value, int childIndex) { + ProcessorNode thisNode = currNode; + ProcessorNode childNode = (ProcessorNode) thisNode.children().get(childIndex); + currNode = childNode; + try { + childNode.process(key, value); + } finally { + currNode = thisNode; + } + } + +} diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java new file mode 100644 index 0000000000000..c0b09f622986b --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -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 org.apache.kafka.test; + +import org.apache.kafka.streams.StreamingMetrics; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateRestoreCallback; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; + +import java.io.File; +import java.util.HashMap; +import java.util.Map; + +public class MockProcessorContext implements ProcessorContext { + + private final KStreamTestDriver driver; + private final Serializer serializer; + private final Deserializer deserializer; + + private Map storeMap = new HashMap<>(); + + long timestamp = -1L; + + public MockProcessorContext(KStreamTestDriver driver, Serializer serializer, Deserializer deserializer) { + this.driver = driver; + this.serializer = serializer; + this.deserializer = deserializer; + } + + public void setTime(long timestamp) { + this.timestamp = timestamp; + } + + public int id() { + return -1; + } + + @Override + public boolean joinable() { + return true; + } + + @Override + public Serializer keySerializer() { + return serializer; + } + + @Override + public Serializer valueSerializer() { + return serializer; + } + + @Override + public Deserializer keyDeserializer() { + return deserializer; + } + + @Override + public Deserializer valueDeserializer() { + return deserializer; + } + + @Override + public File stateDir() { + throw new UnsupportedOperationException("stateDir() not supported."); + } + + @Override + public StreamingMetrics metrics() { + throw new UnsupportedOperationException("metrics() not supported."); + } + + @Override + public void register(StateStore store, StateRestoreCallback func) { + if (func != null) throw new UnsupportedOperationException("StateRestoreCallback not supported."); + storeMap.put(store.name(), store); + } + + @Override + public StateStore getStateStore(String name) { + return storeMap.get(name); + } + + @Override + public void schedule(long interval) { + throw new UnsupportedOperationException("schedule() not supported"); + } + + @Override + @SuppressWarnings("unchecked") + public void forward(K key, V value) { + driver.forward(key, value); + } + + @Override + @SuppressWarnings("unchecked") + public void forward(K key, V value, int childIndex) { + driver.forward(key, value, childIndex); + } + + @Override + public void commit() { + throw new UnsupportedOperationException("commit() not supported."); + } + + @Override + public String topic() { + throw new UnsupportedOperationException("topic() not supported."); + } + + @Override + public int partition() { + throw new UnsupportedOperationException("partition() not supported."); + } + + @Override + public long offset() { + throw new UnsupportedOperationException("offset() not supported."); + } + + @Override + public long timestamp() { + return this.timestamp; + } + +} diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java new file mode 100644 index 0000000000000..f1aa16782d5f4 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.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.test; + +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +import java.util.ArrayList; + +public class MockProcessorSupplier implements ProcessorSupplier { + + public final ArrayList processed = new ArrayList<>(); + public final ArrayList punctuated = new ArrayList<>(); + + @Override + public Processor get() { + return new MockProcessor(); + } + + public class MockProcessor implements Processor { + + @Override + public void init(ProcessorContext context) { + // do nothing + } + + @Override + public void process(K key, V value) { + processed.add(key + ":" + value); + } + + @Override + public void punctuate(long streamTime) { + punctuated.add(streamTime); + } + + @Override + public void close() { + // do nothing + } + + } +} diff --git a/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java b/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java new file mode 100644 index 0000000000000..cf0202e187554 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/MockSourceNode.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.test; + + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.streams.processor.internals.SourceNode; + +import java.util.ArrayList; +import java.util.concurrent.atomic.AtomicInteger; + +public class MockSourceNode extends SourceNode { + + public static final String NAME = "MOCK-SOURCE-"; + public static final AtomicInteger INDEX = new AtomicInteger(1); + + public int numReceived = 0; + public final ArrayList keys = new ArrayList<>(); + public final ArrayList values = new ArrayList<>(); + + public MockSourceNode(Deserializer keyDeserializer, Deserializer valDeserializer) { + super(NAME + INDEX.getAndIncrement(), keyDeserializer, valDeserializer); + } + + @Override + public void process(K key, V value) { + this.numReceived++; + this.keys.add(key); + this.values.add(value); + } +} diff --git a/streams/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java b/streams/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java new file mode 100644 index 0000000000000..274e7b5bd7d4c --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/MockTimestampExtractor.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.test; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.streams.processor.TimestampExtractor; + +/* Extract the timestamp as the offset of the record */ +public class MockTimestampExtractor implements TimestampExtractor { + + @Override + public long extract(ConsumerRecord record) { + return record.offset(); + } +} diff --git a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java new file mode 100644 index 0000000000000..8eb2c62441920 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java @@ -0,0 +1,336 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.MockConsumer; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.clients.producer.MockProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.StreamingConfig; +import org.apache.kafka.streams.StreamingMetrics; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.TopologyBuilder; +import org.apache.kafka.streams.processor.internals.ProcessorTopology; +import org.apache.kafka.streams.processor.internals.StreamTask; +import org.apache.kafka.streams.state.KeyValueStore; + +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.Queue; +import java.util.concurrent.atomic.AtomicLong; + +/** + * This class makes it easier to write tests to verify the behavior of topologies created with a {@link TopologyBuilder}. + * You can test simple topologies that have a single processor, or very complex topologies that have multiple sources, processors, + * and sinks. And because it starts with a {@link TopologyBuilder}, you can create topologies specific to your tests or you + * can use and test code you already have that uses a builder to create topologies. Best of all, the class works without a real + * Kafka broker, so the tests execute very quickly with very little overhead. + *

                  + * Using the ProcessorTopologyTestDriver in tests is easy: simply instantiate the driver with a {@link StreamingConfig} and a + * TopologyBuilder, use the driver to supply an input message to the topology, and then use the driver to read and verify any + * messages output by the topology. + *

                  + * Although the driver doesn't use a real Kafka broker, it does simulate Kafka {@link org.apache.kafka.clients.consumer.Consumer}s + * and {@link org.apache.kafka.clients.producer.Producer}s that read and write raw {@code byte[]} messages. You can either deal + * with messages that have {@code byte[]} keys and values, or you can supply the {@link Serializer}s and {@link Deserializer}s + * that the driver can use to convert the keys and values into objects. + * + *

                  Driver setup

                  + *

                  + * In order to create a ProcessorTopologyTestDriver instance, you need a TopologyBuilder and a {@link StreamingConfig}. The + * configuration needs to be representative of what you'd supply to the real topology, so that means including several key + * properties. For example, the following code fragment creates a configuration that specifies a local Kafka broker list + * (which is needed but not used), a timestamp extractor, and default serializers and deserializers for string keys and values: + * + *

                  + * StringSerializer strSerializer = new StringSerializer();
                  + * StringDeserializer strDeserializer = new StringDeserializer();
                  + * Properties props = new Properties();
                  + * props.setProperty(StreamingConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9091");
                  + * props.setProperty(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, CustomTimestampExtractor.class.getName());
                  + * props.setProperty(StreamingConfig.KEY_SERIALIZER_CLASS_CONFIG, strSerializer.getClass().getName());
                  + * props.setProperty(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG, strDeserializer.getClass().getName());
                  + * props.setProperty(StreamingConfig.VALUE_SERIALIZER_CLASS_CONFIG, strSerializer.getClass().getName());
                  + * props.setProperty(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG, strDeserializer.getClass().getName());
                  + * StreamingConfig config = new StreamingConfig(props);
                  + * TopologyBuilder builder = ...
                  + * ProcessorTopologyTestDriver driver = new ProcessorTopologyTestDriver(config, builder);
                  + * 
                  + * + *

                  Processing messages

                  + *

                  + * Your test can supply new input records on any of the topics that the topology's sources consume. Here's an example of an + * input message on the topic named {@code input-topic}: + * + *

                  + * driver.process("input-topic", "key1", "value1", strSerializer, strSerializer);
                  + * 
                  + * + * Immediately, the driver will pass the input message through to the appropriate source that consumes the named topic, + * and will invoke the processor(s) downstream of the source. If your topology's processors forward messages to sinks, + * your test can then consume these output messages to verify they match the expected outcome. For example, if our topology + * should have generated 2 messages on {@code output-topic-1} and 1 message on {@code output-topic-2}, then our test can + * obtain these messages using the {@link #readOutput(String, Deserializer, Deserializer)} method: + * + *
                  + * ProducerRecord record1 = driver.readOutput("output-topic-1", strDeserializer, strDeserializer);
                  + * ProducerRecord record2 = driver.readOutput("output-topic-1", strDeserializer, strDeserializer);
                  + * ProducerRecord record3 = driver.readOutput("output-topic-2", strDeserializer, strDeserializer);
                  + * 
                  + * + * Again, our example topology generates messages with string keys and values, so we supply our string deserializer instance + * for use on both the keys and values. Your test logic can then verify whether these output records are correct. + *

                  + * Finally, when completed, make sure your tests {@link #close()} the driver to release all resources and + * {@link org.apache.kafka.streams.processor.Processor}s. + * + *

                  Processor state

                  + *

                  + * Some processors use Kafka {@link StateStore state storage}, so this driver class provides the {@link #getStateStore(String)} + * and {@link #getKeyValueStore(String)} methods so that your tests can check the underlying state store(s) used by your + * topology's processors. In our previous example, after we supplied a single input message and checked the three output messages, + * our test could also check the key value store to verify the processor correctly added, removed, or updated internal state. + * Or, our test might have pre-populated some state before submitting the input message, and verified afterward that the + * processor(s) correctly updated the state. + */ +public class ProcessorTopologyTestDriver { + + private final Serializer bytesSerializer = new ByteArraySerializer(); + + private final int id; + private final ProcessorTopology topology; + private final StreamTask task; + private final MockConsumer consumer; + private final MockProducer producer; + private final MockConsumer restoreStateConsumer; + private final Map partitionsByTopic = new HashMap<>(); + private final Map offsetsByTopicPartition = new HashMap<>(); + private final Map>> outputRecordsByTopic = new HashMap<>(); + + /** + * Create a new test driver instance. + * @param config the streaming configuration for the topology + * @param builder the topology builder that will be used to create the topology instance + * @param storeNames the optional names of the state stores that are used by the topology + */ + public ProcessorTopologyTestDriver(StreamingConfig config, TopologyBuilder builder, String... storeNames) { + id = 0; + topology = builder.build(); + + // Set up the consumer and producer ... + consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + producer = new MockProducer<>(true, bytesSerializer, bytesSerializer); + restoreStateConsumer = createRestoreConsumer(id, storeNames); + + // Set up all of the topic+partition information and subscribe the consumer to each ... + for (String topic : topology.sourceTopics()) { + TopicPartition tp = new TopicPartition(topic, 1); + consumer.assign(Collections.singletonList(tp)); + partitionsByTopic.put(topic, tp); + offsetsByTopicPartition.put(tp, new AtomicLong()); + } + + task = new StreamTask(id, + consumer, + producer, + restoreStateConsumer, + partitionsByTopic.values(), + topology, + config, + new StreamingMetrics() { + @Override + public Sensor addLatencySensor(String scopeName, String entityName, String operationName, String... tags) { + return null; + } + + @Override + public void recordLatency(Sensor sensor, long startNs, long endNs) { + // do nothing + } + }); + } + + /** + * Send an input message with the given key and value on the specified topic to the topology, and then commit the messages. + * + * @param topicName the name of the topic on which the message is to be sent + * @param key the raw message key + * @param value the raw message value + */ + public void process(String topicName, byte[] key, byte[] value) { + TopicPartition tp = partitionsByTopic.get(topicName); + if (tp == null) { + throw new IllegalArgumentException("Unexpected topic: " + topicName); + } + // Add the record ... + long offset = offsetsByTopicPartition.get(tp).incrementAndGet(); + task.addRecords(tp, records(new ConsumerRecord(tp.topic(), tp.partition(), offset, key, value))); + producer.clear(); + // Process the record ... + task.process(); + task.commit(); + // Capture all the records sent to the producer ... + for (ProducerRecord record : producer.history()) { + Queue> outputRecords = outputRecordsByTopic.get(record.topic()); + if (outputRecords == null) { + outputRecords = new LinkedList<>(); + outputRecordsByTopic.put(record.topic(), outputRecords); + } + outputRecords.add(record); + } + } + + /** + * Send an input message with the given key and value on the specified topic to the topology. + * + * @param topicName the name of the topic on which the message is to be sent + * @param key the raw message key + * @param value the raw message value + * @param keySerializer the serializer for the key + * @param valueSerializer the serializer for the value + */ + public void process(String topicName, K key, V value, Serializer keySerializer, Serializer valueSerializer) { + process(topicName, keySerializer.serialize(topicName, key), valueSerializer.serialize(topicName, value)); + } + + /** + * Read the next record from the given topic. These records were output by the topology during the previous calls to + * {@link #process(String, byte[], byte[])}. + * + * @param topic the name of the topic + * @return the next record on that topic, or null if there is no record available + */ + public ProducerRecord readOutput(String topic) { + Queue> outputRecords = outputRecordsByTopic.get(topic); + if (outputRecords == null) return null; + return outputRecords.poll(); + } + + /** + * Read the next record from the given topic. These records were output by the topology during the previous calls to + * {@link #process(String, byte[], byte[])}. + * + * @param topic the name of the topic + * @param keyDeserializer the deserializer for the key type + * @param valueDeserializer the deserializer for the value type + * @return the next record on that topic, or null if there is no record available + */ + public ProducerRecord readOutput(String topic, Deserializer keyDeserializer, Deserializer valueDeserializer) { + ProducerRecord record = readOutput(topic); + if (record == null) return null; + K key = keyDeserializer.deserialize(record.topic(), record.key()); + V value = valueDeserializer.deserialize(record.topic(), record.value()); + return new ProducerRecord(record.topic(), record.partition(), key, value); + } + + private Iterable> records(ConsumerRecord record) { + return Collections.singleton(record); + } + + /** + * Get the {@link StateStore} with the given name. The name should have been supplied via + * {@link #ProcessorTopologyTestDriver(StreamingConfig, TopologyBuilder, String...) this object's constructor}, and is + * presumed to be used by a Processor within the topology. + *

                  + * This is often useful in test cases to pre-populate the store before the test case instructs the topology to + * {@link #process(String, byte[], byte[]) process an input message}, and/or to check the store afterward. + * + * @param name the name of the store + * @return the state store, or null if no store has been registered with the given name + * @see #getKeyValueStore(String) + */ + public StateStore getStateStore(String name) { + return task.context().getStateStore(name); + } + + /** + * Get the {@link KeyValueStore} with the given name. The name should have been supplied via + * {@link #ProcessorTopologyTestDriver(StreamingConfig, TopologyBuilder, String...) this object's constructor}, and is + * presumed to be used by a Processor within the topology. + *

                  + * This is often useful in test cases to pre-populate the store before the test case instructs the topology to + * {@link #process(String, byte[], byte[]) process an input message}, and/or to check the store afterward. + *

                  + * + * @param name the name of the store + * @return the key value store, or null if no {@link KeyValueStore} has been registered with the given name + * @see #getStateStore(String) + */ + @SuppressWarnings("unchecked") + public KeyValueStore getKeyValueStore(String name) { + StateStore store = getStateStore(name); + return store instanceof KeyValueStore ? (KeyValueStore) getStateStore(name) : null; + } + + /** + * Close the driver, its topology, and all processors. + */ + public void close() { + task.close(); + } + + /** + * Utility method that creates the {@link MockConsumer} used for restoring state, which should not be done by this + * driver object unless this method is overwritten with a functional consumer. + * + * @param id the ID of the stream task + * @param storeNames the names of the stores that this + * @return the mock consumer; never null + */ + protected MockConsumer createRestoreConsumer(int id, String... storeNames) { + MockConsumer consumer = new MockConsumer(OffsetResetStrategy.LATEST) { + @Override + public synchronized void seekToEnd(TopicPartition... partitions) { + // do nothing ... + } + + @Override + public synchronized void seekToBeginning(TopicPartition... partitions) { + // do nothing ... + } + + @Override + public synchronized long position(TopicPartition partition) { + // do nothing ... + return 0L; + } + }; + // For each store name ... + for (String storeName : storeNames) { + String topicName = storeName; + // Set up the restore-state topic ... + // consumer.subscribe(new TopicPartition(topicName, 1)); + // Set up the partition that matches the ID (which is what ProcessorStateManager expects) ... + List partitionInfos = new ArrayList<>(); + partitionInfos.add(new PartitionInfo(topicName, id, null, null, null)); + consumer.updatePartitions(topicName, partitionInfos); + consumer.updateEndOffsets(Collections.singletonMap(new TopicPartition(topicName, id), 0L)); + } + return consumer; + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java b/streams/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java new file mode 100644 index 0000000000000..04c8f61daca00 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java @@ -0,0 +1,104 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streams.kstream.Window; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.kstream.KeyValue; +import org.apache.kafka.streams.kstream.WindowSupplier; +import org.apache.kafka.streams.kstream.internals.FilteredIterator; +import org.apache.kafka.streams.processor.internals.Stamped; + +import java.util.Iterator; +import java.util.LinkedList; + +public class UnlimitedWindowDef implements WindowSupplier { + + private final String name; + + public UnlimitedWindowDef(String name) { + this.name = name; + } + + public String name() { + return name; + } + + public Window get() { + return new UnlimitedWindow(); + } + + public class UnlimitedWindow implements Window { + + private final LinkedList>> list = new LinkedList<>(); + + @Override + public void init(ProcessorContext context) { + context.register(this, null); + } + + @Override + public Iterator find(final K key, long timestamp) { + return find(key, Long.MIN_VALUE, timestamp); + } + + @Override + public Iterator findAfter(final K key, long timestamp) { + return find(key, timestamp, Long.MAX_VALUE); + } + + @Override + public Iterator findBefore(final K key, long timestamp) { + return find(key, Long.MIN_VALUE, Long.MAX_VALUE); + } + + private Iterator find(final K key, final long startTime, final long endTime) { + return new FilteredIterator>>(list.iterator()) { + protected V filter(Stamped> item) { + if (item.value.key.equals(key) && startTime <= item.timestamp && item.timestamp <= endTime) + return item.value.value; + else + return null; + } + }; + } + + @Override + public void put(K key, V value, long timestamp) { + list.add(new Stamped<>(KeyValue.pair(key, value), timestamp)); + } + + @Override + public String name() { + return name; + } + + @Override + public void flush() { + } + + @Override + public void close() { + } + + @Override + public boolean persistent() { + return false; + } + } +} diff --git a/system_test/README.txt b/system_test/README.txt index 0e469e373c9d9..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,20 +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. / $ ./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. 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 - 4. To turn on debugging, update system_test/logging.conf by changing the level in handlers session from INFO to DEBUG. - # ========================== # Adding Test Case # ========================== 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/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 c0117c64cbb76..48f9ff6b2810f 100644 --- a/system_test/mirror_maker_testsuite/mirror_maker_test.py +++ b/system_test/mirror_maker_testsuite/mirror_maker_test.py @@ -315,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_5001_properties.json b/system_test/mirror_maker_testsuite/testcase_15001/testcase_15001_properties.json similarity index 95% rename from system_test/mirror_maker_testsuite/testcase_15001/testcase_5001_properties.json rename to system_test/mirror_maker_testsuite/testcase_15001/testcase_15001_properties.json index 287cab9a9b6b9..9dd3477e70a53 100644 --- a/system_test/mirror_maker_testsuite/testcase_15001/testcase_5001_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_15001/testcase_15001_properties.json @@ -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", diff --git a/system_test/mirror_maker_testsuite/testcase_15002/testcase_5002_properties.json b/system_test/mirror_maker_testsuite/testcase_15002/testcase_15002_properties.json similarity index 94% rename from system_test/mirror_maker_testsuite/testcase_15002/testcase_5002_properties.json rename to system_test/mirror_maker_testsuite/testcase_15002/testcase_15002_properties.json index 5457eb1d26526..d6495e54d6845 100644 --- a/system_test/mirror_maker_testsuite/testcase_15002/testcase_5002_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_15002/testcase_15002_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", diff --git a/system_test/mirror_maker_testsuite/testcase_15003/testcase_5003_properties.json b/system_test/mirror_maker_testsuite/testcase_15003/testcase_15003_properties.json similarity index 94% rename from system_test/mirror_maker_testsuite/testcase_15003/testcase_5003_properties.json rename to system_test/mirror_maker_testsuite/testcase_15003/testcase_15003_properties.json index 98fefee5838a2..842c70eaf33fd 100644 --- a/system_test/mirror_maker_testsuite/testcase_15003/testcase_5003_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_15003/testcase_15003_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", diff --git a/system_test/mirror_maker_testsuite/testcase_15004/testcase_5004_properties.json b/system_test/mirror_maker_testsuite/testcase_15004/testcase_15004_properties.json similarity index 94% rename from system_test/mirror_maker_testsuite/testcase_15004/testcase_5004_properties.json rename to system_test/mirror_maker_testsuite/testcase_15004/testcase_15004_properties.json index 6067b1263d0a5..48864e61afb9c 100644 --- a/system_test/mirror_maker_testsuite/testcase_15004/testcase_5004_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_15004/testcase_15004_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", diff --git a/system_test/mirror_maker_testsuite/testcase_15005/testcase_5005_properties.json b/system_test/mirror_maker_testsuite/testcase_15005/testcase_15005_properties.json similarity index 95% rename from system_test/mirror_maker_testsuite/testcase_15005/testcase_5005_properties.json rename to system_test/mirror_maker_testsuite/testcase_15005/testcase_15005_properties.json index 58483ad375cd1..92b2a6b4ffeb8 100644 --- a/system_test/mirror_maker_testsuite/testcase_15005/testcase_5005_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_15005/testcase_15005_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", diff --git a/system_test/mirror_maker_testsuite/testcase_15006/testcase_5006_properties.json b/system_test/mirror_maker_testsuite/testcase_15006/testcase_15006_properties.json similarity index 95% rename from system_test/mirror_maker_testsuite/testcase_15006/testcase_5006_properties.json rename to system_test/mirror_maker_testsuite/testcase_15006/testcase_15006_properties.json index 1d9190ce7c1b5..7d5019c6aa7fe 100644 --- a/system_test/mirror_maker_testsuite/testcase_15006/testcase_5006_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_15006/testcase_15006_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", 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 4a0da6ea45201..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", 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 d74e97de642ba..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", 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 e33acf192769d..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", 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 5c39bcf50527d..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", 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 697af656fb58d..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", 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 a610a60d2b18c..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", diff --git a/system_test/offset_management_testsuite/config/server.properties b/system_test/offset_management_testsuite/config/server.properties index 2b988f86052a7..b6de5289f484a 100644 --- a/system_test/offset_management_testsuite/config/server.properties +++ b/system_test/offset_management_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/offset_management_testsuite/offset_management_test.py b/system_test/offset_management_testsuite/offset_management_test.py index 12b5cd25140e1..aa389105aa427 100644 --- a/system_test/offset_management_testsuite/offset_management_test.py +++ b/system_test/offset_management_testsuite/offset_management_test.py @@ -290,6 +290,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/offset_management_testsuite/testcase_7001/testcase_7001_properties.json b/system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json index 02af3e86357db..1f0b7180a9d0a 100644 --- a/system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json +++ b/system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json @@ -30,7 +30,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -41,7 +41,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -52,7 +52,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -63,7 +63,7 @@ "entity_id": "4", "port": "9094", "broker.id": "4", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_4_logs", "default.replication.factor": "3", "num.partitions": "5", 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 index 41ec6e49272f1..9efbd9da7c4b2 100644 --- 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 @@ -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=3 replica.lag.time.max.ms=10000 replica.lag.max.messages=4000 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 index 727e23701d6c2..d4bf702554aa8 100644 --- 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 @@ -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=3 replica.lag.time.max.ms=10000 replica.lag.max.messages=4000 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 index e6fbbe1e0532e..e6e06bef378ab 100644 --- 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 @@ -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=3 replica.lag.time.max.ms=10000 replica.lag.max.messages=4000 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 index fee65bce63564..2cb03e4fbec45 100644 --- 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 @@ -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=3 replica.lag.time.max.ms=10000 replica.lag.max.messages=4000 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 index fdab69b41dbc9..c5866a2ecc42e 100644 --- a/system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json +++ b/system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json @@ -30,7 +30,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -41,7 +41,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -52,7 +52,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -63,7 +63,7 @@ "entity_id": "4", "port": "9094", "broker.id": "4", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_4_logs", "default.replication.factor": "3", "num.partitions": "5", diff --git a/system_test/replication_testsuite/config/server.properties b/system_test/replication_testsuite/config/server.properties index 6becbab60e394..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 diff --git a/system_test/replication_testsuite/replica_basic_test.py b/system_test/replication_testsuite/replica_basic_test.py index 660006cc253bb..16a24a407051a 100644 --- a/system_test/replication_testsuite/replica_basic_test.py +++ b/system_test/replication_testsuite/replica_basic_test.py @@ -451,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 eaaa4ed869995..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", 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 0ffbf67084b67..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", 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 e2fb579b9c0b1..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", 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 62fbe089fb9fe..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", 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 02ad59d085e3c..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", 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 b64304f4d0e4e..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", 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 e8507099a8418..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", 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 47217cf66c758..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", 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 3ddaad422bc86..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", 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 62381518a1b52..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", 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 69203c3f5b412..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", 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 6bc54ced174b0..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", 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 4b3cfe70f6178..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", 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 a913a6c8d2de0..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", 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 55b0bdbb1e81c..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", 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 608ed87132189..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", 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 f1e9caf373079..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", 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 26fa2ac751bea..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", 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 2398722977419..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,7 +50,7 @@ "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" diff --git a/system_test/replication_testsuite/testcase_10101/testcase_0101_properties.json b/system_test/replication_testsuite/testcase_10101/testcase_10101_properties.json similarity index 94% rename from system_test/replication_testsuite/testcase_10101/testcase_0101_properties.json rename to system_test/replication_testsuite/testcase_10101/testcase_10101_properties.json index 85e4b61751b12..3f8e5870799b7 100644 --- a/system_test/replication_testsuite/testcase_10101/testcase_0101_properties.json +++ b/system_test/replication_testsuite/testcase_10101/testcase_10101_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": "20480", "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": "20480", "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": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_10102/testcase_0102_properties.json b/system_test/replication_testsuite/testcase_10102/testcase_10102_properties.json similarity index 94% rename from system_test/replication_testsuite/testcase_10102/testcase_0102_properties.json rename to system_test/replication_testsuite/testcase_10102/testcase_10102_properties.json index 0d2f59fbcaa30..c96352d5e18e6 100644 --- a/system_test/replication_testsuite/testcase_10102/testcase_0102_properties.json +++ b/system_test/replication_testsuite/testcase_10102/testcase_10102_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 : 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": "20480", "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": "20480", "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": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_10103/testcase_0103_properties.json b/system_test/replication_testsuite/testcase_10103/testcase_10103_properties.json similarity index 94% rename from system_test/replication_testsuite/testcase_10103/testcase_0103_properties.json rename to system_test/replication_testsuite/testcase_10103/testcase_10103_properties.json index 34acfa9d5c684..55fa39e148818 100644 --- a/system_test/replication_testsuite/testcase_10103/testcase_0103_properties.json +++ b/system_test/replication_testsuite/testcase_10103/testcase_10103_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 : 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": "20480", "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": "20480", "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": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_10104/testcase_0104_properties.json b/system_test/replication_testsuite/testcase_10104/testcase_10104_properties.json similarity index 94% rename from system_test/replication_testsuite/testcase_10104/testcase_0104_properties.json rename to system_test/replication_testsuite/testcase_10104/testcase_10104_properties.json index 4145345fc433a..15827eb994ae8 100644 --- a/system_test/replication_testsuite/testcase_10104/testcase_0104_properties.json +++ b/system_test/replication_testsuite/testcase_10104/testcase_10104_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 : 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": "20480", "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": "20480", "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": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_10105/testcase_0105_properties.json b/system_test/replication_testsuite/testcase_10105/testcase_10105_properties.json similarity index 94% rename from system_test/replication_testsuite/testcase_10105/testcase_0105_properties.json rename to system_test/replication_testsuite/testcase_10105/testcase_10105_properties.json index 2eecc768e7e97..d1fa1ade07c53 100644 --- a/system_test/replication_testsuite/testcase_10105/testcase_0105_properties.json +++ b/system_test/replication_testsuite/testcase_10105/testcase_10105_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 : 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": "20480", "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": "20480", "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": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_10106/testcase_0106_properties.json b/system_test/replication_testsuite/testcase_10106/testcase_10106_properties.json similarity index 94% rename from system_test/replication_testsuite/testcase_10106/testcase_0106_properties.json rename to system_test/replication_testsuite/testcase_10106/testcase_10106_properties.json index 744174eab1c82..675c76f656efa 100644 --- a/system_test/replication_testsuite/testcase_10106/testcase_0106_properties.json +++ b/system_test/replication_testsuite/testcase_10106/testcase_10106_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 : 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": "20480", "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": "20480", "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": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", diff --git a/system_test/replication_testsuite/testcase_10107/testcase_0107_properties.json b/system_test/replication_testsuite/testcase_10107/testcase_10107_properties.json similarity index 94% rename from system_test/replication_testsuite/testcase_10107/testcase_0107_properties.json rename to system_test/replication_testsuite/testcase_10107/testcase_10107_properties.json index e881b138c1fc0..afc221cdc8bf7 100644 --- a/system_test/replication_testsuite/testcase_10107/testcase_0107_properties.json +++ b/system_test/replication_testsuite/testcase_10107/testcase_10107_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 : 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": "20480", "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": "20480", "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": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", diff --git a/system_test/replication_testsuite/testcase_10108/testcase_0108_properties.json b/system_test/replication_testsuite/testcase_10108/testcase_10108_properties.json similarity index 94% rename from system_test/replication_testsuite/testcase_10108/testcase_0108_properties.json rename to system_test/replication_testsuite/testcase_10108/testcase_10108_properties.json index 7b48fdb441ad5..5df72f3d52a6c 100644 --- a/system_test/replication_testsuite/testcase_10108/testcase_0108_properties.json +++ b/system_test/replication_testsuite/testcase_10108/testcase_10108_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 : 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": "20480", "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": "20480", "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": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", diff --git a/system_test/replication_testsuite/testcase_10109/testcase_0109_properties.json b/system_test/replication_testsuite/testcase_10109/testcase_10109_properties.json similarity index 94% rename from system_test/replication_testsuite/testcase_10109/testcase_0109_properties.json rename to system_test/replication_testsuite/testcase_10109/testcase_10109_properties.json index a98ae0390bf55..9b156681ec485 100644 --- a/system_test/replication_testsuite/testcase_10109/testcase_0109_properties.json +++ b/system_test/replication_testsuite/testcase_10109/testcase_10109_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 : 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": "20480", "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": "20480", "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": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", diff --git a/system_test/replication_testsuite/testcase_10110/testcase_0110_properties.json b/system_test/replication_testsuite/testcase_10110/testcase_10110_properties.json similarity index 100% rename from system_test/replication_testsuite/testcase_10110/testcase_0110_properties.json rename to system_test/replication_testsuite/testcase_10110/testcase_10110_properties.json diff --git a/system_test/replication_testsuite/testcase_10131/testcase_0131_properties.json b/system_test/replication_testsuite/testcase_10131/testcase_10131_properties.json similarity index 100% rename from system_test/replication_testsuite/testcase_10131/testcase_0131_properties.json rename to system_test/replication_testsuite/testcase_10131/testcase_10131_properties.json diff --git a/system_test/replication_testsuite/testcase_10132/testcase_0132_properties.json b/system_test/replication_testsuite/testcase_10132/testcase_10132_properties.json similarity index 100% rename from system_test/replication_testsuite/testcase_10132/testcase_0132_properties.json rename to system_test/replication_testsuite/testcase_10132/testcase_10132_properties.json diff --git a/system_test/replication_testsuite/testcase_10133/testcase_0133_properties.json b/system_test/replication_testsuite/testcase_10133/testcase_10133_properties.json similarity index 100% rename from system_test/replication_testsuite/testcase_10133/testcase_0133_properties.json rename to system_test/replication_testsuite/testcase_10133/testcase_10133_properties.json diff --git a/system_test/replication_testsuite/testcase_10134/testcase_0134_properties.json b/system_test/replication_testsuite/testcase_10134/testcase_10134_properties.json similarity index 100% rename from system_test/replication_testsuite/testcase_10134/testcase_0134_properties.json rename to system_test/replication_testsuite/testcase_10134/testcase_10134_properties.json 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 c6cf17ea69073..0000000000000 --- a/system_test/testcase_to_run.json +++ /dev/null @@ -1,5 +0,0 @@ -{ - "ReplicaBasicTest" : [ - "testcase_1" - ] -} 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 1093b660ebd0c..a9b73f7f3831a 100644 --- a/system_test/utils/kafka_system_test_utils.py +++ b/system_test/utils/kafka_system_test_utils.py @@ -436,6 +436,7 @@ def generate_overriden_props_files(testsuitePathname, testcaseEnv, systemTestEnv 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"] @@ -2400,6 +2401,9 @@ 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" 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/testcase_env.py b/system_test/utils/testcase_env.py index b3c29105c0434..1d2fb5762f531 100644 --- a/system_test/utils/testcase_env.py +++ b/system_test/utils/testcase_env.py @@ -28,68 +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 console-consumer entities - # key: entity_id - # val: ppid of console consumer associated to that entity_id - # { 0: 12345, 1: 12389, ... } - 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, ... } - 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/tests/.gitignore b/tests/.gitignore new file mode 100644 index 0000000000000..b218b83c4edee --- /dev/null +++ b/tests/.gitignore @@ -0,0 +1,11 @@ +Vagrantfile.local + +.idea/ + +*.pyc +*.ipynb + +.DS_Store + +.ducktape +results/ diff --git a/tests/MANIFEST.in b/tests/MANIFEST.in new file mode 100644 index 0000000000000..cda3ed54ef32d --- /dev/null +++ b/tests/MANIFEST.in @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +recursive-include kafkatest/services/templates *.properties diff --git a/tests/README.md b/tests/README.md new file mode 100644 index 0000000000000..82d31daad5bae --- /dev/null +++ b/tests/README.md @@ -0,0 +1,132 @@ +System Integration & Performance Testing +======================================== + +This directory contains Kafka system integration and performance tests. +[ducktape](https://github.com/confluentinc/ducktape) is used to run the tests. +(ducktape is a distributed testing framework which provides test runner, +result reporter and utilities to pull up and tear down services.) + +Local Quickstart +---------------- +This quickstart will help you run the Kafka system tests on your local machine. +For a tutorial on how to setup and run the Kafka system tests, see +https://cwiki.apache.org/confluence/display/KAFKA/tutorial+-+set+up+and+run+Kafka+system+tests+with+ducktape + +* Install Virtual Box from [https://www.virtualbox.org/](https://www.virtualbox.org/) (run `$ vboxmanage --version` to check if it's installed). +* Install Vagrant >= 1.6.4 from [http://www.vagrantup.com/](http://www.vagrantup.com/) (run `vagrant --version` to check if it's installed). +* Install system test dependiences, including ducktape, a command-line tool and library for testing distributed systems. + + $ python tests/setup.py develop + +* Run the bootstrap script to set up Vagrant for testing + + $ tests/bootstrap-test-env.sh + +* Bring up the test cluster + + $ vagrant up + +* Build the desired branch of Kafka + + $ git checkout $BRANCH + $ gradle # (only if necessary) + $ ./gradlew jar + +* Run the system tests using ducktape: + + $ cd tests + $ ducktape kafkatest/tests + +EC2 Quickstart +-------------- +This quickstart will help you run the Kafka system tests on EC2. In this setup, all logic is run +on EC2 and none on your local machine. + +There are a lot of steps here, but the basic goals are to create one distinguished EC2 instance that +will be our "test driver", and to set up the security groups and iam role so that the test driver +can create, destroy, and run ssh commands on any number of "workers". + +As a convention, we'll use "kafkatest" in most names, but you can use whatever name you want. + +Preparation +----------- +In these steps, we will create an IAM role which has permission to create and destroy EC2 instances, +set up a keypair used for ssh access to the test driver and worker machines, and create a security group to allow the test driver and workers to all communicate via TCP. + +* [Create an IAM role](http://docs.aws.amazon.com/IAM/latest/UserGuide/Using_SettingUpUser.html#Using_CreateUser_console). We'll give this role the ability to launch or kill additional EC2 machines. + - Create role "kafkatest-master" + - Role type: Amazon EC2 + - Attach policy: AmazonEC2FullAccess (this will allow our test-driver to create and destroy EC2 instances) + +* If you haven't already, [set up a keypair to use for SSH access](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/ec2-key-pairs.html). For the purpose +of this quickstart, let's say the keypair name is kafkatest, and you've saved the private key in kafktest.pem + +* Next, create a security group called "kafkatest". + - After creating the group, inbound rules: allow SSH on port 22 from anywhere; also, allow access on all ports (0-65535) from other machines in the kafkatest group. + +Create the Test Driver +---------------------- +* Launch a new test driver machine + - OS: Ubuntu server is recommended + - Instance type: t2.medium is easily enough since this machine is just a driver + - Instance details: Most defaults are fine. + - IAM role -> kafkatest-master + - Tagging the instance with a useful name is recommended. + - Security group -> 'kafkatest' + + +* Once the machine is started, upload the SSH key to your test driver: + + $ scp -i /path/to/kafkatest.pem \ + /path/to/kafkatest.pem ubuntu@public.hostname.amazonaws.com:kafkatest.pem + +* Grab the public hostname/IP (available for example by navigating to your EC2 dashboard and viewing running instances) of your test driver and SSH into it: + + $ ssh -i /path/to/kafkatest.pem ubuntu@public.hostname.amazonaws.com + +Set Up the Test Driver +---------------------- +The following steps assume you have ssh'd into +the test driver machine. + +* Start by making sure you're up to date, and install git and ducktape: + + $ sudo apt-get update && sudo apt-get -y upgrade && sudo apt-get install -y git + $ pip install ducktape + +* Get Kafka: + + $ git clone https://git-wip-us.apache.org/repos/asf/kafka.git kafka + +* Install some dependencies: + + $ cd kafka + $ kafka/vagrant/aws/aws-init.sh + $ . ~/.bashrc + +* An example Vagrantfile.local has been created by aws-init.sh which looks something like: + + # Vagrantfile.local + ec2_instance_type = "..." # Pick something appropriate for your + # test. Note that the default m3.medium has + # a small disk. + num_zookeepers = 0 + num_kafka = 0 + num_workers = 9 + ec2_keypair_name = 'kafkatest' + ec2_keypair_file = '/home/ubuntu/kafkatest.pem' + ec2_security_groups = ['kafkatest'] + ec2_region = 'us-west-2' + ec2_ami = "ami-29ebb519" + +* Start up the instances (note we have found bringing up machines in parallel can cause errors on aws): + + $ vagrant up --provider=aws --no-provision --no-parallel && vagrant provision + +* Now you should be able to run tests: + + $ cd kafka/tests + $ ducktape kafkatest/tests + +* To halt your workers without destroying persistent state, run `vagrant halt`. Run `vagrant destroy -f` to destroy all traces of your workers. + diff --git a/tests/bootstrap-test-env.sh b/tests/bootstrap-test-env.sh new file mode 100755 index 0000000000000..2a276287161c2 --- /dev/null +++ b/tests/bootstrap-test-env.sh @@ -0,0 +1,94 @@ +#!/usr/bin/env 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. + +# This script automates the process of setting up a local machine for running Kafka system tests + +# Helper function which prints version numbers so they can be compared lexically or numerically +function version { echo "$@" | awk -F. '{ printf("%03d%03d%03d%03d\n", $1,$2,$3,$4); }'; } + +base_dir=`dirname $0`/.. +cd $base_dir + +echo "Checking Virtual Box installation..." +bad_vb=false +if [ -z `vboxmanage --version` ]; then + echo "It appears that Virtual Box is not installed. Please install and try again (see https://www.virtualbox.org/ for details)" + bad_vb=true +else + echo "Virtual Box looks good." +fi + +echo "Checking Vagrant installation..." +vagrant_version=`vagrant --version | egrep -o "[0-9]+\.[0-9]+\.[0-9]+"` +bad_vagrant=false +if [ "$(version $vagrant_version)" -lt "$(version 1.6.4)" ]; then + echo "Found Vagrant version $vagrant_version. Please upgrade to 1.6.4 or higher (see http://www.vagrantup.com for details)" + bad_vagrant=true +else + echo "Vagrant installation looks good." +fi + +if [ "x$bad_vagrant" == "xtrue" -o "x$bad_vb" == "xtrue" ]; then + exit 1 +fi + +echo "Checking for necessary Vagrant plugins..." +install_hostmanager=false +hostmanager_version=`vagrant plugin list | grep vagrant-hostmanager | egrep -o "[0-9]+\.[0-9]+\.[0-9]+"` +if [ -z "$hostmanager_version" ]; then + install_hostmanager=true +elif [ "$hostmanager_version" != "1.5.0" ]; then + echo "You have the wrong version of vagrant plugin vagrant-hostmanager. Uninstalling..." + vagrant plugin uninstall vagrant-hostmanager + install_hostmanager=true +fi +if [ "x$install_hostmanager" == "xtrue" ]; then + vagrant plugin install vagrant-hostmanager --plugin-version 1.5.0 +fi + +echo "Creating and packaging a reusable base box for Vagrant..." +vagrant/package-base-box.sh + +# Set up Vagrantfile.local if necessary +if [ ! -e Vagrantfile.local ]; then + echo "Creating Vagrantfile.local..." + cp vagrant/system-test-Vagrantfile.local Vagrantfile.local +else + echo "Found an existing Vagrantfile.local. Keeping without overwriting..." +fi + +# Sanity check contents of Vagrantfile.local +echo "Checking Vagrantfile.local..." +vagrantfile_ok=true +num_brokers=`egrep -o "num_brokers\s*=\s*[0-9]+" Vagrantfile.local | cut -d '=' -f 2 | xargs` +num_zookeepers=`egrep -o "num_zookeepers\s*=\s*[0-9]+" Vagrantfile.local | cut -d '=' -f 2 | xargs` +num_workers=`egrep -o "num_workers\s*=\s*[0-9]+" Vagrantfile.local | cut -d '=' -f 2 | xargs` +if [ "x$num_brokers" == "x" -o "$num_brokers" != 0 ]; then + echo "Vagrantfile.local: bad num_brokers. Update to: num_brokers = 0" + vagrantfile_ok=false +fi +if [ "x$num_zookeepers" == "x" -o "$num_zookeepers" != 0 ]; then + echo "Vagrantfile.local: bad num_zookeepers. Update to: num_zookeepers = 0" + vagrantfile_ok=false +fi +if [ "x$num_workers" == "x" -o "$num_workers" == 0 ]; then + echo "Vagrantfile.local: bad num_workers (size of test cluster). Set num_workers high enough to run your tests." + vagrantfile_ok=false +fi + +if [ "$vagrantfile_ok" == "true" ]; then + echo "Vagrantfile.local looks good." +fi diff --git a/tests/kafkatest/__init__.py b/tests/kafkatest/__init__.py new file mode 100644 index 0000000000000..28d269bc4de87 --- /dev/null +++ b/tests/kafkatest/__init__.py @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + diff --git a/tests/kafkatest/sanity_checks/__init__.py b/tests/kafkatest/sanity_checks/__init__.py new file mode 100644 index 0000000000000..91eacc92e8be9 --- /dev/null +++ b/tests/kafkatest/sanity_checks/__init__.py @@ -0,0 +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. \ No newline at end of file diff --git a/tests/kafkatest/sanity_checks/test_console_consumer.py b/tests/kafkatest/sanity_checks/test_console_consumer.py new file mode 100644 index 0000000000000..a9c4d53f0ed40 --- /dev/null +++ b/tests/kafkatest/sanity_checks/test_console_consumer.py @@ -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. + +from ducktape.tests.test import Test +from ducktape.utils.util import wait_until +from ducktape.mark import parametrize +from ducktape.mark import matrix + +from kafkatest.services.zookeeper import ZookeeperService +from kafkatest.services.kafka import KafkaService +from kafkatest.services.console_consumer import ConsoleConsumer +from kafkatest.utils.remote_account import line_count, file_exists + +import time + +class ConsoleConsumerTest(Test): + """Sanity checks on console consumer service class.""" + def __init__(self, test_context): + super(ConsoleConsumerTest, self).__init__(test_context) + + self.topic = "topic" + self.zk = ZookeeperService(test_context, num_nodes=1) + + def setUp(self): + self.zk.start() + + @parametrize(security_protocol='SSL', new_consumer=True) + @matrix(security_protocol=['PLAINTEXT'], new_consumer=[False, True]) + def test_lifecycle(self, security_protocol, new_consumer): + self.kafka = KafkaService(self.test_context, num_nodes=1, zk=self.zk, + security_protocol=security_protocol, + topics={self.topic: {"partitions": 1, "replication-factor": 1}}) + self.kafka.start() + + t0 = time.time() + self.consumer = ConsoleConsumer(self.test_context, num_nodes=1, kafka=self.kafka, topic=self.topic, security_protocol=security_protocol, new_consumer=new_consumer) + self.consumer.start() + node = self.consumer.nodes[0] + + wait_until(lambda: self.consumer.alive(node), + timeout_sec=10, backoff_sec=.2, err_msg="Consumer was too slow to start") + self.logger.info("consumer started in %s seconds " % str(time.time() - t0)) + + # Verify that log output is happening + wait_until(lambda: file_exists(node, ConsoleConsumer.LOG_FILE), timeout_sec=10, + err_msg="Timed out waiting for logging to start.") + assert line_count(node, ConsoleConsumer.LOG_FILE) > 0 + + # Verify no consumed messages + assert line_count(node, ConsoleConsumer.STDOUT_CAPTURE) == 0 + + self.consumer.stop_node(node) diff --git a/tests/kafkatest/sanity_checks/test_mirror_maker.py b/tests/kafkatest/sanity_checks/test_mirror_maker.py new file mode 100644 index 0000000000000..0b0eca3a37f8f --- /dev/null +++ b/tests/kafkatest/sanity_checks/test_mirror_maker.py @@ -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. + + +from ducktape.tests.test import Test + +from kafkatest.services.zookeeper import ZookeeperService +from kafkatest.services.kafka import KafkaService +from kafkatest.services.console_consumer import ConsoleConsumer +from kafkatest.services.verifiable_producer import VerifiableProducer +from kafkatest.services.mirror_maker import MirrorMaker + + +class TestMirrorMakerService(Test): + """Sanity checks on mirror maker service class.""" + def __init__(self, test_context): + super(TestMirrorMakerService, self).__init__(test_context) + + self.topic = "topic" + self.source_zk = ZookeeperService(test_context, num_nodes=1) + self.target_zk = ZookeeperService(test_context, num_nodes=1) + + self.source_kafka = KafkaService(test_context, num_nodes=1, zk=self.source_zk, + topics={self.topic: {"partitions": 1, "replication-factor": 1}}) + self.target_kafka = KafkaService(test_context, num_nodes=1, zk=self.target_zk, + topics={self.topic: {"partitions": 1, "replication-factor": 1}}) + + self.num_messages = 1000 + # This will produce to source kafka cluster + self.producer = VerifiableProducer(test_context, num_nodes=1, kafka=self.source_kafka, topic=self.topic, + max_messages=self.num_messages, throughput=1000) + + # Use a regex whitelist to check that the start command is well-formed in this case + self.mirror_maker = MirrorMaker(test_context, num_nodes=1, source=self.source_kafka, target=self.target_kafka, + whitelist=".*", consumer_timeout_ms=2000) + + # This will consume from target kafka cluster + self.consumer = ConsoleConsumer(test_context, num_nodes=1, kafka=self.target_kafka, topic=self.topic, + consumer_timeout_ms=1000) + + def setUp(self): + # Source cluster + self.source_zk.start() + self.source_kafka.start() + + # Target cluster + self.target_zk.start() + self.target_kafka.start() + + def test_end_to_end(self): + """ + Test end-to-end behavior under non-failure conditions. + + Setup: two single node Kafka clusters, each connected to its own single node zookeeper cluster. + One is source, and the other is target. Single-node mirror maker mirrors from source to target. + + - Start mirror maker. + - Produce a small number of messages to the source cluster. + - Consume messages from target. + - Verify that number of consumed messages matches the number produced. + """ + self.mirror_maker.start() + # Check that consumer_timeout_ms setting made it to config file + self.mirror_maker.nodes[0].account.ssh( + "grep \"consumer\.timeout\.ms\" %s" % MirrorMaker.CONSUMER_CONFIG, allow_fail=False) + + self.producer.start() + self.producer.wait(10) + self.consumer.start() + self.consumer.wait(10) + + num_consumed = len(self.consumer.messages_consumed[1]) + num_produced = self.producer.num_acked + assert num_produced == self.num_messages, "num_produced: %d, num_messages: %d" % (num_produced, self.num_messages) + assert num_produced == num_consumed, "num_produced: %d, num_consumed: %d" % (num_produced, num_consumed) + + self.mirror_maker.stop() + diff --git a/tests/kafkatest/services/__init__.py b/tests/kafkatest/services/__init__.py new file mode 100644 index 0000000000000..ebc9bb3a9a03f --- /dev/null +++ b/tests/kafkatest/services/__init__.py @@ -0,0 +1,15 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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 diff --git a/tests/kafkatest/services/console_consumer.py b/tests/kafkatest/services/console_consumer.py new file mode 100644 index 0000000000000..2f1e70e199e22 --- /dev/null +++ b/tests/kafkatest/services/console_consumer.py @@ -0,0 +1,210 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from ducktape.services.background_thread import BackgroundThreadService +from ducktape.utils.util import wait_until +from kafkatest.utils.security_config import SecurityConfig + +import os +import subprocess + + +def is_int(msg): + """Default method used to check whether text pulled from console consumer is a message. + + return int or None + """ + try: + return int(msg) + except: + return None + +""" +0.8.2.1 ConsoleConsumer options + +The console consumer is a tool that reads data from Kafka and outputs it to standard output. +Option Description +------ ----------- +--blacklist Blacklist of topics to exclude from + consumption. +--consumer.config Consumer config properties file. +--csv-reporter-enabled If set, the CSV metrics reporter will + be enabled +--delete-consumer-offsets If specified, the consumer path in + zookeeper is deleted when starting up +--formatter The name of a class to use for + formatting kafka messages for + display. (default: kafka.tools. + DefaultMessageFormatter) +--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. +--max-messages The maximum number of messages to + consume before exiting. If not set, + consumption is continual. +--metrics-dir If csv-reporter-enable is set, and + this parameter isset, the csv + metrics will be outputed here +--property +--skip-message-on-error If there is an error when processing a + message, skip it instead of halt. +--topic The topic id to consume on. +--whitelist Whitelist of topics to include for + consumption. +--zookeeper REQUIRED: The connection string for + the zookeeper connection in the form + host:port. Multiple URLS can be + given to allow fail-over. +""" + + +class ConsoleConsumer(BackgroundThreadService): + # Root directory for persistent output + PERSISTENT_ROOT = "/mnt/console_consumer" + STDOUT_CAPTURE = os.path.join(PERSISTENT_ROOT, "console_consumer.stdout") + STDERR_CAPTURE = os.path.join(PERSISTENT_ROOT, "console_consumer.stderr") + LOG_DIR = os.path.join(PERSISTENT_ROOT, "logs") + LOG_FILE = os.path.join(LOG_DIR, "console_consumer.log") + LOG4J_CONFIG = os.path.join(PERSISTENT_ROOT, "tools-log4j.properties") + CONFIG_FILE = os.path.join(PERSISTENT_ROOT, "console_consumer.properties") + + logs = { + "consumer_stdout": { + "path": STDOUT_CAPTURE, + "collect_default": False}, + "consumer_stderr": { + "path": STDERR_CAPTURE, + "collect_default": False}, + "consumer_log": { + "path": LOG_FILE, + "collect_default": True} + } + + def __init__(self, context, num_nodes, kafka, topic, security_protocol=None, new_consumer=None, message_validator=None, from_beginning=True, consumer_timeout_ms=None): + """ + Args: + context: standard context + num_nodes: number of nodes to use (this should be 1) + kafka: kafka service + topic: consume from this topic + security_protocol: security protocol for Kafka connections + new_consumer: use new Kafka consumer if True + message_validator: function which returns message or None + from_beginning: consume from beginning if True, else from the end + consumer_timeout_ms: corresponds to consumer.timeout.ms. consumer process ends if time between + successively consumed messages exceeds this timeout. Setting this and + waiting for the consumer to stop is a pretty good way to consume all messages + in a topic. + """ + super(ConsoleConsumer, self).__init__(context, num_nodes) + self.kafka = kafka + self.new_consumer = new_consumer + self.args = { + 'topic': topic, + } + + self.consumer_timeout_ms = consumer_timeout_ms + + self.from_beginning = from_beginning + self.message_validator = message_validator + self.messages_consumed = {idx: [] for idx in range(1, num_nodes + 1)} + + # Process client configuration + self.prop_file = self.render('console_consumer.properties', consumer_timeout_ms=self.consumer_timeout_ms) + + # Add security properties to the config. If security protocol is not specified, + # use the default in the template properties. + self.security_config = SecurityConfig(security_protocol, self.prop_file) + self.security_protocol = self.security_config.security_protocol + if self.new_consumer is None: + self.new_consumer = self.security_protocol == SecurityConfig.SSL + if self.security_protocol == SecurityConfig.SSL and not self.new_consumer: + raise Exception("SSL protocol is supported only with the new consumer") + self.prop_file += str(self.security_config) + + @property + def start_cmd(self): + args = self.args.copy() + args['zk_connect'] = self.kafka.zk.connect_setting() + args['stdout'] = ConsoleConsumer.STDOUT_CAPTURE + args['stderr'] = ConsoleConsumer.STDERR_CAPTURE + args['config_file'] = ConsoleConsumer.CONFIG_FILE + + cmd = "export LOG_DIR=%s;" % ConsoleConsumer.LOG_DIR + cmd += " export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%s\";" % ConsoleConsumer.LOG4J_CONFIG + cmd += " /opt/kafka/bin/kafka-console-consumer.sh --topic %(topic)s" \ + " --consumer.config %(config_file)s" % args + + if self.new_consumer: + cmd += " --new-consumer --bootstrap-server %s" % self.kafka.bootstrap_servers() + else: + cmd += " --zookeeper %(zk_connect)s" % args + if self.from_beginning: + cmd += " --from-beginning" + + cmd += " 2>> %(stderr)s | tee -a %(stdout)s &" % args + return cmd + + def pids(self, node): + try: + cmd = "ps ax | grep -i console_consumer | grep java | grep -v grep | awk '{print $1}'" + pid_arr = [pid for pid in node.account.ssh_capture(cmd, allow_fail=True, callback=int)] + return pid_arr + except (subprocess.CalledProcessError, ValueError) as e: + return [] + + def alive(self, node): + return len(self.pids(node)) > 0 + + def _worker(self, idx, node): + node.account.ssh("mkdir -p %s" % ConsoleConsumer.PERSISTENT_ROOT, allow_fail=False) + + self.logger.info("console_consumer.properties:") + self.logger.info(self.prop_file) + node.account.create_file(ConsoleConsumer.CONFIG_FILE, self.prop_file) + self.security_config.setup_node(node) + + # Create and upload log properties + log_config = self.render('tools_log4j.properties', log_file=ConsoleConsumer.LOG_FILE) + node.account.create_file(ConsoleConsumer.LOG4J_CONFIG, log_config) + + # Run and capture output + cmd = self.start_cmd + self.logger.debug("Console consumer %d command: %s", idx, cmd) + for line in node.account.ssh_capture(cmd, allow_fail=False): + msg = line.strip() + if self.message_validator is not None: + msg = self.message_validator(msg) + if msg is not None: + self.messages_consumed[idx].append(msg) + + def start_node(self, node): + super(ConsoleConsumer, self).start_node(node) + + def stop_node(self, node): + node.account.kill_process("java", allow_fail=True) + wait_until(lambda: not self.alive(node), timeout_sec=10, backoff_sec=.2, + err_msg="Timed out waiting for consumer to stop.") + + def clean_node(self, node): + if self.alive(node): + self.logger.warn("%s %s was still alive at cleanup time. Killing forcefully..." % + (self.__class__.__name__, node.account)) + node.account.kill_process("java", clean_shutdown=False, allow_fail=True) + node.account.ssh("rm -rf %s" % ConsoleConsumer.PERSISTENT_ROOT, allow_fail=False) + self.security_config.clean_node(node) + diff --git a/tests/kafkatest/services/copycat.py b/tests/kafkatest/services/copycat.py new file mode 100644 index 0000000000000..ea5753bc5c500 --- /dev/null +++ b/tests/kafkatest/services/copycat.py @@ -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. + +from ducktape.services.service import Service +from ducktape.utils.util import wait_until +import subprocess, signal + + +class CopycatServiceBase(Service): + """Base class for Copycat services providing some common settings and functionality""" + + logs = { + "kafka_log": { + "path": "/mnt/copycat.log", + "collect_default": True}, + } + + def __init__(self, context, num_nodes, kafka, files): + super(CopycatServiceBase, self).__init__(context, num_nodes) + self.kafka = kafka + self.files = files + + def pids(self, node): + """Return process ids for Copycat processes.""" + try: + return [pid for pid in node.account.ssh_capture("cat /mnt/copycat.pid", callback=int)] + except: + return [] + + def stop_node(self, node, clean_shutdown=True): + pids = self.pids(node) + sig = signal.SIGTERM if clean_shutdown else signal.SIGKILL + + for pid in pids: + node.account.signal(pid, sig, allow_fail=False) + for pid in pids: + wait_until(lambda: not node.account.alive(pid), timeout_sec=10, err_msg="Copycat standalone process took too long to exit") + + node.account.ssh("rm -f /mnt/copycat.pid", allow_fail=False) + + def restart(self): + # We don't want to do any clean up here, just restart the process + for node in self.nodes: + self.stop_node(node) + self.start_node(node) + + def clean_node(self, node): + if len(self.pids(node)) > 0: + self.logger.warn("%s %s was still alive at cleanup time. Killing forcefully..." % + (self.__class__.__name__, node.account)) + for pid in self.pids(node): + node.account.signal(pid, signal.SIGKILL, allow_fail=False) + node.account.ssh("rm -rf /mnt/copycat.pid /mnt/copycat.log /mnt/copycat.properties /mnt/copycat-connector.properties " + " ".join(self.files), allow_fail=False) + + +class CopycatStandaloneService(CopycatServiceBase): + """Runs Copycat in standalone mode.""" + + def __init__(self, context, kafka, files): + super(CopycatStandaloneService, self).__init__(context, 1, kafka, files) + + def set_configs(self, config_template, connector_config_template): + """ + Set configurations for the worker and the connector to run on + it. These are not provided in the constructor because the worker + config generally needs access to ZK/Kafka services to + create the configuration. + """ + self.config_template = config_template + self.connector_config_template = connector_config_template + + # For convenience since this service only makes sense with a single node + @property + def node(self): + return self.nodes[0] + + def start_node(self, node): + node.account.create_file("/mnt/copycat.properties", self.config_template) + node.account.create_file("/mnt/copycat-connector.properties", self.connector_config_template) + + self.logger.info("Starting Copycat standalone process") + with node.account.monitor_log("/mnt/copycat.log") as monitor: + node.account.ssh("/opt/kafka/bin/copycat-standalone.sh /mnt/copycat.properties /mnt/copycat-connector.properties " + + "1>> /mnt/copycat.log 2>> /mnt/copycat.log & echo $! > /mnt/copycat.pid") + monitor.wait_until('Copycat started', timeout_sec=10, err_msg="Never saw message indicating Copycat finished startup") + + if len(self.pids(node)) == 0: + raise RuntimeError("No process ids recorded") + + + +class CopycatDistributedService(CopycatServiceBase): + """Runs Copycat in distributed mode.""" + + def __init__(self, context, num_nodes, kafka, files, offsets_topic="copycat-offsets"): + super(CopycatDistributedService, self).__init__(context, num_nodes, kafka, files) + self.offsets_topic = offsets_topic + + def set_configs(self, config_template, connector_config_template): + """ + Set configurations for the worker and the connector to run on + it. These are not provided in the constructor because the worker + config generally needs access to ZK/Kafka services to + create the configuration. + """ + self.config_template = config_template + self.connector_config_template = connector_config_template + + def start_node(self, node): + node.account.create_file("/mnt/copycat.properties", self.config_template) + node.account.create_file("/mnt/copycat-connector.properties", self.connector_config_template) + + self.logger.info("Starting Copycat standalone process") + with node.account.monitor_log("/mnt/copycat.log") as monitor: + node.account.ssh("/opt/kafka/bin/copycat-distributed.sh /mnt/copycat.properties /mnt/copycat-connector.properties " + + "1>> /mnt/copycat.log 2>> /mnt/copycat.log & echo $! > /mnt/copycat.pid") + monitor.wait_until('Copycat started', timeout_sec=10, err_msg="Never saw message indicating Copycat finished startup") + + if len(self.pids(node)) == 0: + raise RuntimeError("No process ids recorded") + diff --git a/tests/kafkatest/services/kafka.py b/tests/kafkatest/services/kafka.py new file mode 100644 index 0000000000000..2c200f37927ef --- /dev/null +++ b/tests/kafkatest/services/kafka.py @@ -0,0 +1,244 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from ducktape.services.service import Service +from ducktape.utils.util import wait_until +from kafkatest.utils.security_config import SecurityConfig + +import json +import re +import signal +import time + + +class KafkaService(Service): + + logs = { + "kafka_log": { + "path": "/mnt/kafka.log", + "collect_default": True}, + "kafka_data": { + "path": "/mnt/kafka-logs", + "collect_default": False} + } + + def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAINTEXT, interbroker_security_protocol=SecurityConfig.PLAINTEXT, topics=None): + """ + :type context + :type zk: ZookeeperService + :type topics: dict + """ + super(KafkaService, self).__init__(context, num_nodes) + self.zk = zk + if security_protocol == SecurityConfig.SSL or interbroker_security_protocol == SecurityConfig.SSL: + self.security_config = SecurityConfig(SecurityConfig.SSL) + else: + self.security_config = SecurityConfig(SecurityConfig.PLAINTEXT) + self.security_protocol = security_protocol + self.interbroker_security_protocol = interbroker_security_protocol + self.port = 9092 if security_protocol == SecurityConfig.PLAINTEXT else 9093 + self.topics = topics + + def start(self): + super(KafkaService, self).start() + + # Create topics if necessary + if self.topics is not None: + for topic, topic_cfg in self.topics.items(): + if topic_cfg is None: + topic_cfg = {} + + topic_cfg["topic"] = topic + self.create_topic(topic_cfg) + + def start_node(self, node): + props_file = self.render('kafka.properties', node=node, broker_id=self.idx(node), + port = self.port, security_protocol = self.security_protocol, + interbroker_security_protocol=self.interbroker_security_protocol) + self.logger.info("kafka.properties:") + self.logger.info(props_file) + node.account.create_file("/mnt/kafka.properties", props_file) + self.security_config.setup_node(node) + + cmd = "/opt/kafka/bin/kafka-server-start.sh /mnt/kafka.properties 1>> /mnt/kafka.log 2>> /mnt/kafka.log & echo $! > /mnt/kafka.pid" + self.logger.debug("Attempting to start KafkaService on %s with command: %s" % (str(node.account), cmd)) + with node.account.monitor_log("/mnt/kafka.log") as monitor: + node.account.ssh(cmd) + monitor.wait_until("Kafka Server.*started", timeout_sec=30, err_msg="Kafka server didn't finish startup") + if len(self.pids(node)) == 0: + raise Exception("No process ids recorded on node %s" % str(node)) + + def pids(self, node): + """Return process ids associated with running processes on the given node.""" + try: + return [pid for pid in node.account.ssh_capture("cat /mnt/kafka.pid", callback=int)] + except: + return [] + + def signal_node(self, node, sig=signal.SIGTERM): + pids = self.pids(node) + for pid in pids: + node.account.signal(pid, sig) + + def signal_leader(self, topic, partition=0, sig=signal.SIGTERM): + leader = self.leader(topic, partition) + self.signal_node(leader, sig) + + def stop_node(self, node, clean_shutdown=True): + pids = self.pids(node) + sig = signal.SIGTERM if clean_shutdown else signal.SIGKILL + + for pid in pids: + node.account.signal(pid, sig, allow_fail=False) + + node.account.ssh("rm -f /mnt/kafka.pid", allow_fail=False) + + def clean_node(self, node): + node.account.kill_process("kafka", clean_shutdown=False, allow_fail=True) + node.account.ssh("rm -rf /mnt/kafka-logs /mnt/kafka.properties /mnt/kafka.log /mnt/kafka.pid", allow_fail=False) + self.security_config.clean_node(node) + + def create_topic(self, topic_cfg): + node = self.nodes[0] # any node is fine here + self.logger.info("Creating topic %s with settings %s", topic_cfg["topic"], topic_cfg) + + cmd = "/opt/kafka/bin/kafka-topics.sh --zookeeper %(zk_connect)s --create "\ + "--topic %(topic)s --partitions %(partitions)d --replication-factor %(replication)d" % { + 'zk_connect': self.zk.connect_setting(), + 'topic': topic_cfg.get("topic"), + 'partitions': topic_cfg.get('partitions', 1), + 'replication': topic_cfg.get('replication-factor', 1) + } + + if "configs" in topic_cfg.keys() and topic_cfg["configs"] is not None: + for config_name, config_value in topic_cfg["configs"].items(): + cmd += " --config %s=%s" % (config_name, str(config_value)) + + self.logger.info("Running topic creation command...\n%s" % cmd) + node.account.ssh(cmd) + + time.sleep(1) + self.logger.info("Checking to see if topic was properly created...\n%s" % cmd) + for line in self.describe_topic(topic_cfg["topic"]).split("\n"): + self.logger.info(line) + + def describe_topic(self, topic): + node = self.nodes[0] + cmd = "/opt/kafka/bin/kafka-topics.sh --zookeeper %s --topic %s --describe" % \ + (self.zk.connect_setting(), topic) + output = "" + for line in node.account.ssh_capture(cmd): + output += line + return output + + def verify_reassign_partitions(self, reassignment): + """Run the reassign partitions admin tool in "verify" mode + """ + node = self.nodes[0] + json_file = "/tmp/" + str(time.time()) + "_reassign.json" + + # reassignment to json + json_str = json.dumps(reassignment) + json_str = json.dumps(json_str) + + # create command + cmd = "echo %s > %s && " % (json_str, json_file) + cmd += "/opt/kafka/bin/kafka-reassign-partitions.sh "\ + "--zookeeper %(zk_connect)s "\ + "--reassignment-json-file %(reassignment_file)s "\ + "--verify" % {'zk_connect': self.zk.connect_setting(), + 'reassignment_file': json_file} + cmd += " && sleep 1 && rm -f %s" % json_file + + # send command + self.logger.info("Verifying parition reassignment...") + self.logger.debug(cmd) + output = "" + for line in node.account.ssh_capture(cmd): + output += line + + self.logger.debug(output) + + if re.match(".*is in progress.*", output) is not None: + return False + + return True + + def execute_reassign_partitions(self, reassignment): + """Run the reassign partitions admin tool in "verify" mode + """ + node = self.nodes[0] + json_file = "/tmp/" + str(time.time()) + "_reassign.json" + + # reassignment to json + json_str = json.dumps(reassignment) + json_str = json.dumps(json_str) + + # create command + cmd = "echo %s > %s && " % (json_str, json_file) + cmd += "/opt/kafka/bin/kafka-reassign-partitions.sh "\ + "--zookeeper %(zk_connect)s "\ + "--reassignment-json-file %(reassignment_file)s "\ + "--execute" % {'zk_connect': self.zk.connect_setting(), + 'reassignment_file': json_file} + cmd += " && sleep 1 && rm -f %s" % json_file + + # send command + self.logger.info("Executing parition reassignment...") + self.logger.debug(cmd) + output = "" + for line in node.account.ssh_capture(cmd): + output += line + + self.logger.debug("Verify partition reassignment:") + self.logger.debug(output) + + def restart_node(self, node, wait_sec=0, clean_shutdown=True): + """Restart the given node, waiting wait_sec in between stopping and starting up again.""" + self.stop_node(node, clean_shutdown) + time.sleep(wait_sec) + self.start_node(node) + + def leader(self, topic, partition=0): + """ Get the leader replica for the given topic and partition. + """ + cmd = "/opt/kafka/bin/kafka-run-class.sh kafka.tools.ZooKeeperMainWrapper -server %s " \ + % self.zk.connect_setting() + cmd += "get /brokers/topics/%s/partitions/%d/state" % (topic, partition) + self.logger.debug(cmd) + + node = self.nodes[0] + self.logger.debug("Querying zookeeper to find leader replica for topic %s: \n%s" % (cmd, topic)) + partition_state = None + for line in node.account.ssh_capture(cmd): + match = re.match("^({.+})$", line) + if match is not None: + partition_state = match.groups()[0] + break + + if partition_state is None: + raise Exception("Error finding partition state for topic %s and partition %d." % (topic, partition)) + + partition_state = json.loads(partition_state) + self.logger.info(partition_state) + + leader_idx = int(partition_state["leader"]) + self.logger.info("Leader for topic %s and partition %d is now: %d" % (topic, partition, leader_idx)) + return self.get_node(leader_idx) + + def bootstrap_servers(self): + """Get the broker list to connect to Kafka using the specified security protocol + """ + return ','.join([node.account.hostname + ":" + `self.port` for node in self.nodes]) diff --git a/tests/kafkatest/services/kafka_log4j_appender.py b/tests/kafkatest/services/kafka_log4j_appender.py new file mode 100644 index 0000000000000..11369aa8d31cb --- /dev/null +++ b/tests/kafkatest/services/kafka_log4j_appender.py @@ -0,0 +1,61 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from ducktape.services.background_thread import BackgroundThreadService + + +class KafkaLog4jAppender(BackgroundThreadService): + + logs = { + "producer_log": { + "path": "/mnt/kafka_log4j_appender.log", + "collect_default": False} + } + + def __init__(self, context, num_nodes, kafka, topic, max_messages=-1): + super(KafkaLog4jAppender, self).__init__(context, num_nodes) + + self.kafka = kafka + self.topic = topic + self.max_messages = max_messages + + def _worker(self, idx, node): + cmd = self.start_cmd + self.logger.debug("VerifiableKafkaLog4jAppender %d command: %s" % (idx, cmd)) + node.account.ssh(cmd) + + @property + def start_cmd(self): + cmd = "/opt/kafka/bin/kafka-run-class.sh org.apache.kafka.clients.tools.VerifiableLog4jAppender" \ + " --topic %s --broker-list %s" % (self.topic, self.kafka.bootstrap_servers()) + if self.max_messages > 0: + cmd += " --max-messages %s" % str(self.max_messages) + + cmd += " 2>> /mnt/kafka_log4j_appender.log | tee -a /mnt/kafka_log4j_appender.log &" + return cmd + + def stop_node(self, node): + node.account.kill_process("VerifiableKafkaLog4jAppender", allow_fail=False) + if self.worker_threads is None: + return + + # block until the corresponding thread exits + if len(self.worker_threads) >= self.idx(node): + # Need to guard this because stop is preemptively called before the worker threads are added and started + self.worker_threads[self.idx(node) - 1].join() + + def clean_node(self, node): + node.account.kill_process("VerifiableKafkaLog4jAppender", clean_shutdown=False, allow_fail=False) + node.account.ssh("rm -rf /mnt/kafka_log4j_appender.log", allow_fail=False) diff --git a/tests/kafkatest/services/mirror_maker.py b/tests/kafkatest/services/mirror_maker.py new file mode 100644 index 0000000000000..afbed133eb4eb --- /dev/null +++ b/tests/kafkatest/services/mirror_maker.py @@ -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. + +from ducktape.services.service import Service +from ducktape.utils.util import wait_until + +import os +import subprocess + +""" +0.8.2.1 MirrorMaker options + +Option Description +------ ----------- +--abort.on.send.failure +--blacklist Blacklist of topics to mirror. +--consumer.config Embedded consumer config for consuming + from the source cluster. +--consumer.rebalance.listener +--help Print this message. +--message.handler producer. +--message.handler.args +--num.streams (default: 1) +--offset.commit.interval.ms +--producer.config Embedded producer config. +--rebalance.listener.args +--whitelist Whitelist of topics to mirror. +""" + + +class MirrorMaker(Service): + + # Root directory for persistent output + PERSISTENT_ROOT = "/mnt/mirror_maker" + LOG_DIR = os.path.join(PERSISTENT_ROOT, "logs") + LOG_FILE = os.path.join(LOG_DIR, "mirror_maker.log") + LOG4J_CONFIG = os.path.join(PERSISTENT_ROOT, "tools-log4j.properties") + PRODUCER_CONFIG = os.path.join(PERSISTENT_ROOT, "producer.properties") + CONSUMER_CONFIG = os.path.join(PERSISTENT_ROOT, "consumer.properties") + KAFKA_HOME = "/opt/kafka/" + + logs = { + "mirror_maker_log": { + "path": LOG_FILE, + "collect_default": True} + } + + def __init__(self, context, num_nodes, source, target, whitelist=None, blacklist=None, num_streams=1, consumer_timeout_ms=None): + """ + MirrorMaker mirrors messages from one or more source clusters to a single destination cluster. + + Args: + context: standard context + source: source Kafka cluster + target: target Kafka cluster to which data will be mirrored + whitelist: whitelist regex for topics to mirror + blacklist: blacklist regex for topics not to mirror + num_streams: number of consumer threads to create; can be a single int, or a list with + one value per node, allowing num_streams to be the same for each node, + or configured independently per-node + consumer_timeout_ms: consumer stops if t > consumer_timeout_ms elapses between consecutive messages + """ + super(MirrorMaker, self).__init__(context, num_nodes=num_nodes) + + self.consumer_timeout_ms = consumer_timeout_ms + self.num_streams = num_streams + if not isinstance(num_streams, int): + # if not an integer, num_streams should be configured per-node + assert len(num_streams) == num_nodes + self.whitelist = whitelist + self.blacklist = blacklist + self.source = source + self.target = target + + def start_cmd(self, node): + cmd = "export LOG_DIR=%s;" % MirrorMaker.LOG_DIR + cmd += " export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%s\";" % MirrorMaker.LOG4J_CONFIG + cmd += " %s/bin/kafka-run-class.sh kafka.tools.MirrorMaker" % MirrorMaker.KAFKA_HOME + cmd += " --consumer.config %s" % MirrorMaker.CONSUMER_CONFIG + cmd += " --producer.config %s" % MirrorMaker.PRODUCER_CONFIG + if isinstance(self.num_streams, int): + cmd += " --num.streams %d" % self.num_streams + else: + # config num_streams separately on each node + cmd += " --num.streams %d" % self.num_streams[self.idx(node) - 1] + if self.whitelist is not None: + cmd += " --whitelist=\"%s\"" % self.whitelist + if self.blacklist is not None: + cmd += " --blacklist=\"%s\"" % self.blacklist + cmd += " 1>> %s 2>> %s &" % (MirrorMaker.LOG_FILE, MirrorMaker.LOG_FILE) + return cmd + + def pids(self, node): + try: + cmd = "ps ax | grep -i MirrorMaker | grep java | grep -v grep | awk '{print $1}'" + pid_arr = [pid for pid in node.account.ssh_capture(cmd, allow_fail=True, callback=int)] + return pid_arr + except (subprocess.CalledProcessError, ValueError) as e: + return [] + + def alive(self, node): + return len(self.pids(node)) > 0 + + def start_node(self, node): + node.account.ssh("mkdir -p %s" % MirrorMaker.PERSISTENT_ROOT, allow_fail=False) + node.account.ssh("mkdir -p %s" % MirrorMaker.LOG_DIR, allow_fail=False) + + # Create, upload one consumer config file for source cluster + consumer_props = self.render('consumer.properties', zookeeper_connect=self.source.zk.connect_setting()) + node.account.create_file(MirrorMaker.CONSUMER_CONFIG, consumer_props) + + # Create, upload producer properties file for target cluster + producer_props = self.render('producer.properties', broker_list=self.target.bootstrap_servers(), + producer_type="async") + node.account.create_file(MirrorMaker.PRODUCER_CONFIG, producer_props) + + # Create and upload log properties + log_config = self.render('tools_log4j.properties', log_file=MirrorMaker.LOG_FILE) + node.account.create_file(MirrorMaker.LOG4J_CONFIG, log_config) + + # Run mirror maker + cmd = self.start_cmd(node) + self.logger.debug("Mirror maker command: %s", cmd) + node.account.ssh(cmd, allow_fail=False) + wait_until(lambda: self.alive(node), timeout_sec=10, backoff_sec=.5, + err_msg="Mirror maker took to long to start.") + self.logger.debug("Mirror maker is alive") + + def stop_node(self, node): + node.account.kill_process("java", allow_fail=True) + wait_until(lambda: not self.alive(node), timeout_sec=10, backoff_sec=.5, + err_msg="Mirror maker took to long to stop.") + + def clean_node(self, node): + if self.alive(node): + self.logger.warn("%s %s was still alive at cleanup time. Killing forcefully..." % + (self.__class__.__name__, node.account)) + node.account.kill_process("java", clean_shutdown=False, allow_fail=True) + node.account.ssh("rm -rf %s" % MirrorMaker.PERSISTENT_ROOT, allow_fail=False) diff --git a/tests/kafkatest/services/performance/__init__.py b/tests/kafkatest/services/performance/__init__.py new file mode 100644 index 0000000000000..a72e3b792bd9f --- /dev/null +++ b/tests/kafkatest/services/performance/__init__.py @@ -0,0 +1,19 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from performance import PerformanceService +from end_to_end_latency import EndToEndLatencyService +from producer_performance import ProducerPerformanceService +from consumer_performance import ConsumerPerformanceService \ No newline at end of file diff --git a/tests/kafkatest/services/performance/consumer_performance.py b/tests/kafkatest/services/performance/consumer_performance.py new file mode 100644 index 0000000000000..b8eab22471749 --- /dev/null +++ b/tests/kafkatest/services/performance/consumer_performance.py @@ -0,0 +1,155 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from kafkatest.services.performance import PerformanceService +from kafkatest.utils.security_config import SecurityConfig + +import os + + +class ConsumerPerformanceService(PerformanceService): + """ + See ConsumerPerformance.scala as the source of truth on these settings, but for reference: + + "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." + + "broker-list", "A broker list to use for connecting if using the new consumer." + + "topic", "REQUIRED: The topic to consume from." + + "group", "The group id to consume on." + + "fetch-size", "The amount of data to fetch in a single request." + + "from-latest", "If the consumer does not already have an establishedoffset to consume from, + start with the latest message present in the log rather than the earliest message." + + "socket-buffer-size", "The size of the tcp RECV size." + + "threads", "Number of processing threads." + + "num-fetch-threads", "Number of fetcher threads. Defaults to 1" + + "new-consumer", "Use the new consumer implementation." + "consumer.config", "Consumer config properties file." + """ + + # Root directory for persistent output + PERSISTENT_ROOT = "/mnt/consumer_performance" + LOG_DIR = os.path.join(PERSISTENT_ROOT, "logs") + STDOUT_CAPTURE = os.path.join(PERSISTENT_ROOT, "consumer_performance.stdout") + LOG_FILE = os.path.join(LOG_DIR, "consumer_performance.log") + LOG4J_CONFIG = os.path.join(PERSISTENT_ROOT, "tools-log4j.properties") + CONFIG_FILE = os.path.join(PERSISTENT_ROOT, "consumer.properties") + + logs = { + "consumer_performance_output": { + "path": STDOUT_CAPTURE, + "collect_default": True}, + + "consumer_performance_log": { + "path": LOG_FILE, + "collect_default": True} + } + + def __init__(self, context, num_nodes, kafka, security_protocol, topic, messages, new_consumer=False, settings={}): + super(ConsumerPerformanceService, self).__init__(context, num_nodes) + self.kafka = kafka + self.security_config = SecurityConfig(security_protocol) + self.security_protocol = security_protocol + self.topic = topic + self.messages = messages + self.new_consumer = new_consumer + self.settings = settings + + # These less-frequently used settings can be updated manually after instantiation + self.fetch_size = None + self.socket_buffer_size = None + self.threads = None + self.num_fetch_threads = None + self.group = None + self.from_latest = None + + @property + def args(self): + """Dictionary of arguments used to start the Consumer Performance script.""" + args = { + 'topic': self.topic, + 'messages': self.messages, + } + + if self.new_consumer: + args['new-consumer'] = "" + args['broker-list'] = self.kafka.bootstrap_servers() + else: + args['zookeeper'] = self.kafka.zk.connect_setting() + + if self.fetch_size is not None: + args['fetch-size'] = self.fetch_size + + if self.socket_buffer_size is not None: + args['socket-buffer-size'] = self.socket_buffer_size + + if self.threads is not None: + args['threads'] = self.threads + + if self.num_fetch_threads is not None: + args['num-fetch-threads'] = self.num_fetch_threads + + if self.group is not None: + args['group'] = self.group + + if self.from_latest: + args['from-latest'] = "" + + return args + + @property + def start_cmd(self): + cmd = "export LOG_DIR=%s;" % ConsumerPerformanceService.LOG_DIR + cmd += " export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%s\";" % ConsumerPerformanceService.LOG4J_CONFIG + cmd += " /opt/kafka/bin/kafka-consumer-perf-test.sh" + for key, value in self.args.items(): + cmd += " --%s %s" % (key, value) + cmd += " --consumer.config %s" % ConsumerPerformanceService.CONFIG_FILE + + for key, value in self.settings.items(): + cmd += " %s=%s" % (str(key), str(value)) + + cmd += " | tee %s" % ConsumerPerformanceService.STDOUT_CAPTURE + return cmd + + def _worker(self, idx, node): + node.account.ssh("mkdir -p %s" % ConsumerPerformanceService.PERSISTENT_ROOT, allow_fail=False) + + log_config = self.render('tools_log4j.properties', log_file=ConsumerPerformanceService.LOG_FILE) + node.account.create_file(ConsumerPerformanceService.LOG4J_CONFIG, log_config) + node.account.create_file(ConsumerPerformanceService.CONFIG_FILE, str(self.security_config)) + self.security_config.setup_node(node) + + cmd = self.start_cmd + self.logger.debug("Consumer performance %d command: %s", idx, cmd) + last = None + for line in node.account.ssh_capture(cmd): + last = line + # Parse and save the last line's information + parts = last.split(',') + + self.results[idx-1] = { + 'total_mb': float(parts[2]), + 'mbps': float(parts[3]), + 'records_per_sec': float(parts[5]), + } diff --git a/tests/kafkatest/services/performance/end_to_end_latency.py b/tests/kafkatest/services/performance/end_to_end_latency.py new file mode 100644 index 0000000000000..0559a43174e81 --- /dev/null +++ b/tests/kafkatest/services/performance/end_to_end_latency.py @@ -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. + +from kafkatest.services.performance import PerformanceService +from kafkatest.utils.security_config import SecurityConfig + + +class EndToEndLatencyService(PerformanceService): + + logs = { + "end_to_end_latency_log": { + "path": "/mnt/end-to-end-latency.log", + "collect_default": True}, + } + + def __init__(self, context, num_nodes, kafka, security_protocol, topic, num_records, consumer_fetch_max_wait=100, acks=1): + super(EndToEndLatencyService, self).__init__(context, num_nodes) + self.kafka = kafka + self.security_config = SecurityConfig(security_protocol) + self.security_protocol = security_protocol + self.args = { + 'topic': topic, + 'num_records': num_records, + 'consumer_fetch_max_wait': consumer_fetch_max_wait, + 'acks': acks + } + + def _worker(self, idx, node): + args = self.args.copy() + self.security_config.setup_node(node) + if self.security_protocol == SecurityConfig.SSL: + ssl_config_file = SecurityConfig.SSL_DIR + "/security.properties" + node.account.create_file(ssl_config_file, str(self.security_config)) + else: + ssl_config_file = "" + args.update({ + 'zk_connect': self.kafka.zk.connect_setting(), + 'bootstrap_servers': self.kafka.bootstrap_servers(), + 'ssl_config_file': ssl_config_file + }) + + cmd = "/opt/kafka/bin/kafka-run-class.sh kafka.tools.EndToEndLatency "\ + "%(bootstrap_servers)s %(topic)s %(num_records)d "\ + "%(acks)d 20 %(ssl_config_file)s" % args + + cmd += " | tee /mnt/end-to-end-latency.log" + + self.logger.debug("End-to-end latency %d command: %s", idx, cmd) + results = {} + for line in node.account.ssh_capture(cmd): + if line.startswith("Avg latency:"): + results['latency_avg_ms'] = float(line.split()[2]) + if line.startswith("Percentiles"): + results['latency_50th_ms'] = float(line.split()[3][:-1]) + results['latency_99th_ms'] = float(line.split()[6][:-1]) + results['latency_999th_ms'] = float(line.split()[9]) + self.results[idx-1] = results diff --git a/tests/kafkatest/services/performance/performance.py b/tests/kafkatest/services/performance/performance.py new file mode 100644 index 0000000000000..6d286f60faddf --- /dev/null +++ b/tests/kafkatest/services/performance/performance.py @@ -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. + +from ducktape.services.background_thread import BackgroundThreadService + + +class PerformanceService(BackgroundThreadService): + + def __init__(self, context, num_nodes): + super(PerformanceService, self).__init__(context, num_nodes) + self.results = [None] * self.num_nodes + self.stats = [[] for x in range(self.num_nodes)] + + def clean_node(self, node): + node.account.kill_process("java", clean_shutdown=False, allow_fail=True) + node.account.ssh("rm -rf /mnt/*", allow_fail=False) + diff --git a/tests/kafkatest/services/performance/producer_performance.py b/tests/kafkatest/services/performance/producer_performance.py new file mode 100644 index 0000000000000..7a026fcfd7806 --- /dev/null +++ b/tests/kafkatest/services/performance/producer_performance.py @@ -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. + +from kafkatest.services.performance import PerformanceService +from kafkatest.utils.security_config import SecurityConfig + + +class ProducerPerformanceService(PerformanceService): + + logs = { + "producer_performance_log": { + "path": "/mnt/producer-performance.log", + "collect_default": True}, + } + + def __init__(self, context, num_nodes, kafka, security_protocol, topic, num_records, record_size, throughput, settings={}, intermediate_stats=False): + super(ProducerPerformanceService, self).__init__(context, num_nodes) + self.kafka = kafka + self.security_config = SecurityConfig(security_protocol) + self.security_protocol = security_protocol + self.args = { + 'topic': topic, + 'num_records': num_records, + 'record_size': record_size, + 'throughput': throughput + } + self.settings = settings + self.intermediate_stats = intermediate_stats + + def _worker(self, idx, node): + args = self.args.copy() + args.update({'bootstrap_servers': self.kafka.bootstrap_servers()}) + cmd = "/opt/kafka/bin/kafka-run-class.sh org.apache.kafka.clients.tools.ProducerPerformance "\ + "%(topic)s %(num_records)d %(record_size)d %(throughput)d bootstrap.servers=%(bootstrap_servers)s" % args + + self.security_config.setup_node(node) + if self.security_protocol == SecurityConfig.SSL: + self.settings.update(self.security_config.properties) + for key, value in self.settings.items(): + cmd += " %s=%s" % (str(key), str(value)) + cmd += " | tee /mnt/producer-performance.log" + + self.logger.debug("Producer performance %d command: %s", idx, cmd) + + def parse_stats(line): + parts = line.split(',') + return { + 'records': int(parts[0].split()[0]), + 'records_per_sec': float(parts[1].split()[0]), + 'mbps': float(parts[1].split('(')[1].split()[0]), + 'latency_avg_ms': float(parts[2].split()[0]), + 'latency_max_ms': float(parts[3].split()[0]), + 'latency_50th_ms': float(parts[4].split()[0]), + 'latency_95th_ms': float(parts[5].split()[0]), + 'latency_99th_ms': float(parts[6].split()[0]), + 'latency_999th_ms': float(parts[7].split()[0]), + } + last = None + for line in node.account.ssh_capture(cmd): + if self.intermediate_stats: + try: + self.stats[idx-1].append(parse_stats(line)) + except: + # Sometimes there are extraneous log messages + pass + + last = line + try: + self.results[idx-1] = parse_stats(last) + except: + raise Exception("Unable to parse aggregate performance statistics on node %d: %s" % (idx, last)) diff --git a/tests/kafkatest/services/performance/templates/tools_log4j.properties b/tests/kafkatest/services/performance/templates/tools_log4j.properties new file mode 100644 index 0000000000000..ce30d527abc2e --- /dev/null +++ b/tests/kafkatest/services/performance/templates/tools_log4j.properties @@ -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. + +# Define the root logger with appender file +log4j.rootLogger = {{ log_level|default("INFO") }}, FILE + +log4j.appender.FILE=org.apache.log4j.FileAppender +log4j.appender.FILE.File={{ log_file }} +log4j.appender.FILE.ImmediateFlush=true +log4j.appender.FILE.Threshold=debug +# Set the append to false, overwrite +log4j.appender.FILE.Append=false +log4j.appender.FILE.layout=org.apache.log4j.PatternLayout +log4j.appender.FILE.layout.conversionPattern=[%d] %p %m (%c)%n \ No newline at end of file diff --git a/tests/kafkatest/services/templates/console_consumer.properties b/tests/kafkatest/services/templates/console_consumer.properties new file mode 100644 index 0000000000000..7143179748d85 --- /dev/null +++ b/tests/kafkatest/services/templates/console_consumer.properties @@ -0,0 +1,19 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +{% if consumer_timeout_ms is defined and consumer_timeout_ms is not none %} +consumer.timeout.ms={{ consumer_timeout_ms }} +{% endif %} diff --git a/tests/kafkatest/services/templates/consumer.properties b/tests/kafkatest/services/templates/consumer.properties new file mode 100644 index 0000000000000..b8723d14fa1a8 --- /dev/null +++ b/tests/kafkatest/services/templates/consumer.properties @@ -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. +# see kafka.consumer.ConsumerConfig for more details + +zookeeper.connect={{ zookeeper_connect }} +zookeeper.connection.timeout.ms={{ zookeeper_connection_timeout_ms|default(6000) }} +group.id={{ group_id|default('test-consumer-group') }} + +{% if consumer_timeout_ms is defined and consumer_timeout_ms is not none %} +consumer.timeout.ms={{ consumer_timeout_ms }} +{% endif %} diff --git a/tests/kafkatest/services/templates/kafka.properties b/tests/kafkatest/services/templates/kafka.properties new file mode 100644 index 0000000000000..036a8dba6a91e --- /dev/null +++ b/tests/kafkatest/services/templates/kafka.properties @@ -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. +# see kafka.server.KafkaConfig for additional details and defaults + + +broker.id={{ broker_id }} +port=9092 +#host.name=localhost +advertised.host.name={{ node.account.hostname }} +#advertised.port= +{% if security_protocol == interbroker_security_protocol %} +listeners={{ security_protocol }}://:{{ port }} +advertised.listeners={{ security_protocol }}://{{ node.account.hostname }}:{{ port }} +{% else %} +listeners=PLAINTEXT://:9092,SSL://:9093 +advertised.listeners=PLAINTEXT://{{ node.account.hostname }}:9092,SSL://{{ node.account.hostname }}:9093 +{% endif %} +num.network.threads=3 +num.io.threads=8 +socket.send.buffer.bytes=102400 +socket.receive.buffer.bytes=65536 +socket.request.max.bytes=104857600 + +log.dirs=/mnt/kafka-logs +num.partitions=1 +num.recovery.threads.per.data.dir=1 +#log.flush.interval.messages=10000 +#log.flush.interval.ms=1000 +log.retention.hours=168 +#log.retention.bytes=1073741824 +log.segment.bytes=1073741824 +log.retention.check.interval.ms=300000 +log.cleaner.enable=false + +zookeeper.connect={{ zk.connect_setting() }} +zookeeper.connection.timeout.ms=2000 + +security.inter.broker.protocol={{ interbroker_security_protocol }} +ssl.keystore.location=/mnt/ssl/test.keystore.jks +ssl.keystore.password=test-ks-passwd +ssl.key.password=test-key-passwd +ssl.keystore.type=JKS +ssl.truststore.location=/mnt/ssl/test.truststore.jks +ssl.truststore.password=test-ts-passwd +ssl.truststore.type=JKS + diff --git a/tests/kafkatest/services/templates/producer.properties b/tests/kafkatest/services/templates/producer.properties new file mode 100644 index 0000000000000..ede60c8f3229e --- /dev/null +++ b/tests/kafkatest/services/templates/producer.properties @@ -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. +# see kafka.producer.ProducerConfig for more details + +metadata.broker.list={{ broker_list }} +bootstrap.servers = {{ broker_list }} +producer.type={{ producer_type }} # sync or async +compression.codec=none +serializer.class=kafka.serializer.DefaultEncoder + +#partitioner.class= +#compressed.topics= +#queue.buffering.max.ms= +#queue.buffering.max.messages= +#queue.enqueue.timeout.ms= +#batch.num.messages= diff --git a/tests/kafkatest/services/templates/tools_log4j.properties b/tests/kafkatest/services/templates/tools_log4j.properties new file mode 100644 index 0000000000000..ce30d527abc2e --- /dev/null +++ b/tests/kafkatest/services/templates/tools_log4j.properties @@ -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. + +# Define the root logger with appender file +log4j.rootLogger = {{ log_level|default("INFO") }}, FILE + +log4j.appender.FILE=org.apache.log4j.FileAppender +log4j.appender.FILE.File={{ log_file }} +log4j.appender.FILE.ImmediateFlush=true +log4j.appender.FILE.Threshold=debug +# Set the append to false, overwrite +log4j.appender.FILE.Append=false +log4j.appender.FILE.layout=org.apache.log4j.PatternLayout +log4j.appender.FILE.layout.conversionPattern=[%d] %p %m (%c)%n \ No newline at end of file diff --git a/tests/kafkatest/services/templates/zookeeper.properties b/tests/kafkatest/services/templates/zookeeper.properties new file mode 100644 index 0000000000000..e66c53f728875 --- /dev/null +++ b/tests/kafkatest/services/templates/zookeeper.properties @@ -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. +# see kafka.server.KafkaConfig for additional details and defaults + +dataDir=/mnt/zookeeper +clientPort=2181 +maxClientCnxns=0 +initLimit=5 +syncLimit=2 +quorumListenOnAllIPs=true +{% for node in nodes %} +server.{{ loop.index }}={{ node.account.hostname }}:2888:3888 +{% endfor %} \ No newline at end of file diff --git a/tests/kafkatest/services/verifiable_producer.py b/tests/kafkatest/services/verifiable_producer.py new file mode 100644 index 0000000000000..7ae7988b67393 --- /dev/null +++ b/tests/kafkatest/services/verifiable_producer.py @@ -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. + +from ducktape.services.background_thread import BackgroundThreadService +from kafkatest.utils.security_config import SecurityConfig + +import json + + +class VerifiableProducer(BackgroundThreadService): + + CONFIG_FILE = "/mnt/verifiable_producer.properties" + logs = { + "producer_log": { + "path": "/mnt/producer.log", + "collect_default": False} + } + + def __init__(self, context, num_nodes, kafka, topic, security_protocol=None, max_messages=-1, throughput=100000): + super(VerifiableProducer, self).__init__(context, num_nodes) + + self.kafka = kafka + self.topic = topic + self.max_messages = max_messages + self.throughput = throughput + + self.acked_values = [] + self.not_acked_values = [] + + self.prop_file = "" + self.security_config = SecurityConfig(security_protocol, self.prop_file) + self.security_protocol = self.security_config.security_protocol + self.prop_file += str(self.security_config) + + def _worker(self, idx, node): + # Create and upload config file + self.logger.info("verifiable_producer.properties:") + self.logger.info(self.prop_file) + node.account.create_file(VerifiableProducer.CONFIG_FILE, self.prop_file) + self.security_config.setup_node(node) + + cmd = self.start_cmd + self.logger.debug("VerifiableProducer %d command: %s" % (idx, cmd)) + + for line in node.account.ssh_capture(cmd): + line = line.strip() + + data = self.try_parse_json(line) + if data is not None: + + with self.lock: + if data["name"] == "producer_send_error": + data["node"] = idx + self.not_acked_values.append(int(data["value"])) + + elif data["name"] == "producer_send_success": + self.acked_values.append(int(data["value"])) + + @property + def start_cmd(self): + cmd = "/opt/kafka/bin/kafka-verifiable-producer.sh" \ + " --topic %s --broker-list %s" % (self.topic, self.kafka.bootstrap_servers()) + if self.max_messages > 0: + cmd += " --max-messages %s" % str(self.max_messages) + if self.throughput > 0: + cmd += " --throughput %s" % str(self.throughput) + + cmd += " --producer.config %s" % VerifiableProducer.CONFIG_FILE + cmd += " 2>> /mnt/producer.log | tee -a /mnt/producer.log &" + return cmd + + @property + def acked(self): + with self.lock: + return self.acked_values + + @property + def not_acked(self): + with self.lock: + return self.not_acked_values + + @property + def num_acked(self): + with self.lock: + return len(self.acked_values) + + @property + def num_not_acked(self): + with self.lock: + return len(self.not_acked_values) + + def stop_node(self, node): + node.account.kill_process("VerifiableProducer", allow_fail=False) + if self.worker_threads is None: + return + + # block until the corresponding thread exits + if len(self.worker_threads) >= self.idx(node): + # Need to guard this because stop is preemptively called before the worker threads are added and started + self.worker_threads[self.idx(node) - 1].join() + + def clean_node(self, node): + node.account.kill_process("VerifiableProducer", clean_shutdown=False, allow_fail=False) + node.account.ssh("rm -rf /mnt/producer.log /mnt/verifiable_producer.properties", allow_fail=False) + self.security_config.clean_node(node) + + def try_parse_json(self, string): + """Try to parse a string as json. Return None if not parseable.""" + try: + record = json.loads(string) + return record + except ValueError: + self.logger.debug("Could not parse as json: %s" % str(string)) + return None diff --git a/tests/kafkatest/services/zookeeper.py b/tests/kafkatest/services/zookeeper.py new file mode 100644 index 0000000000000..09bec35d4a017 --- /dev/null +++ b/tests/kafkatest/services/zookeeper.py @@ -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. + + +from ducktape.services.service import Service + +import subprocess +import time + + +class ZookeeperService(Service): + + logs = { + "zk_log": { + "path": "/mnt/zk.log", + "collect_default": True} + } + + def __init__(self, context, num_nodes): + """ + :type context + """ + super(ZookeeperService, self).__init__(context, num_nodes) + + def start_node(self, node): + idx = self.idx(node) + self.logger.info("Starting ZK node %d on %s", idx, node.account.hostname) + + node.account.ssh("mkdir -p /mnt/zookeeper") + node.account.ssh("echo %d > /mnt/zookeeper/myid" % idx) + + config_file = self.render('zookeeper.properties') + self.logger.info("zookeeper.properties:") + self.logger.info(config_file) + node.account.create_file("/mnt/zookeeper.properties", config_file) + + node.account.ssh( + "/opt/kafka/bin/zookeeper-server-start.sh /mnt/zookeeper.properties 1>> %(path)s 2>> %(path)s &" + % self.logs["zk_log"]) + + time.sleep(5) # give it some time to start + + def pids(self, node): + try: + cmd = "ps ax | grep -i zookeeper | grep java | grep -v grep | awk '{print $1}'" + pid_arr = [pid for pid in node.account.ssh_capture(cmd, allow_fail=True, callback=int)] + return pid_arr + except (subprocess.CalledProcessError, ValueError) as e: + return [] + + def alive(self, node): + return len(self.pids(node)) > 0 + + def stop_node(self, node): + idx = self.idx(node) + self.logger.info("Stopping %s node %d on %s" % (type(self).__name__, idx, node.account.hostname)) + node.account.kill_process("zookeeper", allow_fail=False) + + def clean_node(self, node): + self.logger.info("Cleaning ZK node %d on %s", self.idx(node), node.account.hostname) + if self.alive(node): + self.logger.warn("%s %s was still alive at cleanup time. Killing forcefully..." % + (self.__class__.__name__, node.account)) + node.account.kill_process("zookeeper", clean_shutdown=False, allow_fail=True) + node.account.ssh("rm -rf /mnt/zookeeper /mnt/zookeeper.properties /mnt/zk.log", allow_fail=False) + + def connect_setting(self): + return ','.join([node.account.hostname + ':2181' for node in self.nodes]) diff --git a/tests/kafkatest/tests/__init__.py b/tests/kafkatest/tests/__init__.py new file mode 100644 index 0000000000000..ebc9bb3a9a03f --- /dev/null +++ b/tests/kafkatest/tests/__init__.py @@ -0,0 +1,15 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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 diff --git a/tests/kafkatest/tests/benchmark_test.py b/tests/kafkatest/tests/benchmark_test.py new file mode 100644 index 0000000000000..7219c0a0ebf78 --- /dev/null +++ b/tests/kafkatest/tests/benchmark_test.py @@ -0,0 +1,244 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from ducktape.services.service import Service +from ducktape.tests.test import Test +from ducktape.mark import parametrize +from ducktape.mark import matrix + +from kafkatest.services.zookeeper import ZookeeperService +from kafkatest.services.kafka import KafkaService +from kafkatest.services.performance import ProducerPerformanceService, EndToEndLatencyService, ConsumerPerformanceService + + +TOPIC_REP_ONE = "topic-replication-factor-one" +TOPIC_REP_THREE = "topic-replication-factor-three" +DEFAULT_RECORD_SIZE = 100 # bytes + + +class Benchmark(Test): + """A benchmark of Kafka producer/consumer performance. This replicates the test + run here: + https://engineering.linkedin.com/kafka/benchmarking-apache-kafka-2-million-writes-second-three-cheap-machines + """ + def __init__(self, test_context): + super(Benchmark, self).__init__(test_context) + self.num_zk = 1 + self.num_brokers = 3 + self.topics = { + TOPIC_REP_ONE: {'partitions': 6, 'replication-factor': 1}, + TOPIC_REP_THREE: {'partitions': 6, 'replication-factor': 3} + } + + self.zk = ZookeeperService(test_context, self.num_zk) + + self.msgs_large = 10000000 + self.batch_size = 8*1024 + self.buffer_memory = 64*1024*1024 + self.msg_sizes = [10, 100, 1000, 10000, 100000] + self.target_data_size = 128*1024*1024 + self.target_data_size_gb = self.target_data_size/float(1024*1024*1024) + + def setUp(self): + self.zk.start() + + def start_kafka(self, security_protocol, interbroker_security_protocol): + self.kafka = KafkaService( + self.test_context, self.num_brokers, + self.zk, security_protocol=security_protocol, + interbroker_security_protocol=interbroker_security_protocol, topics=self.topics) + self.kafka.start() + + @parametrize(acks=1, topic=TOPIC_REP_ONE) + @parametrize(acks=1, topic=TOPIC_REP_THREE) + @parametrize(acks=-1, topic=TOPIC_REP_THREE) + @parametrize(acks=1, topic=TOPIC_REP_THREE, num_producers=3) + @matrix(acks=[1], topic=[TOPIC_REP_THREE], message_size=[10, 100, 1000, 10000, 100000], security_protocol=['PLAINTEXT', 'SSL']) + def test_producer_throughput(self, acks, topic, num_producers=1, message_size=DEFAULT_RECORD_SIZE, security_protocol='PLAINTEXT'): + """ + Setup: 1 node zk + 3 node kafka cluster + Produce ~128MB worth of messages to a topic with 6 partitions. Required acks, topic replication factor, + security protocol and message size are varied depending on arguments injected into this test. + + Collect and return aggregate throughput statistics after all messages have been acknowledged. + (This runs ProducerPerformance.java under the hood) + """ + self.start_kafka(security_protocol, security_protocol) + # Always generate the same total amount of data + nrecords = int(self.target_data_size / message_size) + + self.producer = ProducerPerformanceService( + self.test_context, num_producers, self.kafka, security_protocol=security_protocol, topic=topic, + num_records=nrecords, record_size=message_size, throughput=-1, + settings={ + 'acks': acks, + 'batch.size': self.batch_size, + 'buffer.memory': self.buffer_memory}) + self.producer.run() + return compute_aggregate_throughput(self.producer) + + @parametrize(security_protocol='PLAINTEXT', interbroker_security_protocol='PLAINTEXT') + @matrix(security_protocol=['SSL'], interbroker_security_protocol=['PLAINTEXT', 'SSL']) + def test_long_term_producer_throughput(self, security_protocol, interbroker_security_protocol): + """ + Setup: 1 node zk + 3 node kafka cluster + Produce 10e6 100 byte messages to a topic with 6 partitions, replication-factor 3, and acks=1. + + Collect and return aggregate throughput statistics after all messages have been acknowledged. + + (This runs ProducerPerformance.java under the hood) + """ + self.start_kafka(security_protocol, security_protocol) + self.producer = ProducerPerformanceService( + self.test_context, 1, self.kafka, security_protocol=security_protocol, + topic=TOPIC_REP_THREE, num_records=self.msgs_large, record_size=DEFAULT_RECORD_SIZE, + throughput=-1, settings={'acks': 1, 'batch.size': self.batch_size, 'buffer.memory': self.buffer_memory}, + intermediate_stats=True + ) + self.producer.run() + + summary = ["Throughput over long run, data > memory:"] + data = {} + # FIXME we should be generating a graph too + # Try to break it into 5 blocks, but fall back to a smaller number if + # there aren't even 5 elements + block_size = max(len(self.producer.stats[0]) / 5, 1) + nblocks = len(self.producer.stats[0]) / block_size + + for i in range(nblocks): + subset = self.producer.stats[0][i*block_size:min((i+1)*block_size, len(self.producer.stats[0]))] + if len(subset) == 0: + summary.append(" Time block %d: (empty)" % i) + data[i] = None + else: + records_per_sec = sum([stat['records_per_sec'] for stat in subset])/float(len(subset)) + mb_per_sec = sum([stat['mbps'] for stat in subset])/float(len(subset)) + + summary.append(" Time block %d: %f rec/sec (%f MB/s)" % (i, records_per_sec, mb_per_sec)) + data[i] = throughput(records_per_sec, mb_per_sec) + + self.logger.info("\n".join(summary)) + return data + + + @parametrize(security_protocol='PLAINTEXT', interbroker_security_protocol='PLAINTEXT') + @matrix(security_protocol=['SSL'], interbroker_security_protocol=['PLAINTEXT', 'SSL']) + def test_end_to_end_latency(self, security_protocol, interbroker_security_protocol): + """ + Setup: 1 node zk + 3 node kafka cluster + Produce (acks = 1) and consume 10e3 messages to a topic with 6 partitions and replication-factor 3, + measuring the latency between production and consumption of each message. + + Return aggregate latency statistics. + + (Under the hood, this simply runs EndToEndLatency.scala) + """ + self.start_kafka(security_protocol, interbroker_security_protocol) + self.logger.info("BENCHMARK: End to end latency") + self.perf = EndToEndLatencyService( + self.test_context, 1, self.kafka, + topic=TOPIC_REP_THREE, security_protocol=security_protocol, num_records=10000 + ) + self.perf.run() + return latency(self.perf.results[0]['latency_50th_ms'], self.perf.results[0]['latency_99th_ms'], self.perf.results[0]['latency_999th_ms']) + + @parametrize(new_consumer=True, security_protocol='SSL', interbroker_security_protocol='PLAINTEXT') + @parametrize(new_consumer=True, security_protocol='SSL', interbroker_security_protocol='SSL') + @matrix(new_consumer=[True, False], security_protocol=['PLAINTEXT']) + def test_producer_and_consumer(self, new_consumer, security_protocol, interbroker_security_protocol='PLAINTEXT'): + """ + Setup: 1 node zk + 3 node kafka cluster + Concurrently produce and consume 10e6 messages with a single producer and a single consumer, + using new consumer if new_consumer == True + + Return aggregate throughput statistics for both producer and consumer. + + (Under the hood, this runs ProducerPerformance.java, and ConsumerPerformance.scala) + """ + self.start_kafka(security_protocol, interbroker_security_protocol) + num_records = 10 * 1000 * 1000 # 10e6 + + self.producer = ProducerPerformanceService( + self.test_context, 1, self.kafka, + topic=TOPIC_REP_THREE, security_protocol=security_protocol, + num_records=num_records, record_size=DEFAULT_RECORD_SIZE, throughput=-1, + settings={'acks': 1, 'batch.size': self.batch_size, 'buffer.memory': self.buffer_memory} + ) + self.consumer = ConsumerPerformanceService( + self.test_context, 1, self.kafka, security_protocol, topic=TOPIC_REP_THREE, new_consumer=new_consumer, messages=num_records) + Service.run_parallel(self.producer, self.consumer) + + data = { + "producer": compute_aggregate_throughput(self.producer), + "consumer": compute_aggregate_throughput(self.consumer) + } + summary = [ + "Producer + consumer:", + str(data)] + self.logger.info("\n".join(summary)) + return data + + @parametrize(new_consumer=True, security_protocol='SSL', interbroker_security_protocol='PLAINTEXT') + @parametrize(new_consumer=True, security_protocol='SSL', interbroker_security_protocol='SSL') + @matrix(new_consumer=[True, False], security_protocol=['PLAINTEXT']) + def test_consumer_throughput(self, new_consumer, security_protocol, interbroker_security_protocol='PLAINTEXT', num_consumers=1): + """ + Consume 10e6 100-byte messages with 1 or more consumers from a topic with 6 partitions + (using new consumer iff new_consumer == True), and report throughput. + """ + self.start_kafka(security_protocol, interbroker_security_protocol) + num_records = 10 * 1000 * 1000 # 10e6 + + # seed kafka w/messages + self.producer = ProducerPerformanceService( + self.test_context, 1, self.kafka, + topic=TOPIC_REP_THREE, security_protocol=security_protocol, + num_records=num_records, record_size=DEFAULT_RECORD_SIZE, throughput=-1, + settings={'acks': 1, 'batch.size': self.batch_size, 'buffer.memory': self.buffer_memory} + ) + self.producer.run() + + # consume + self.consumer = ConsumerPerformanceService( + self.test_context, num_consumers, self.kafka, + topic=TOPIC_REP_THREE, security_protocol=security_protocol, new_consumer=new_consumer, messages=num_records) + self.consumer.group = "test-consumer-group" + self.consumer.run() + return compute_aggregate_throughput(self.consumer) + + +def throughput(records_per_sec, mb_per_sec): + """Helper method to ensure uniform representation of throughput data""" + return { + "records_per_sec": records_per_sec, + "mb_per_sec": mb_per_sec + } + + +def latency(latency_50th_ms, latency_99th_ms, latency_999th_ms): + """Helper method to ensure uniform representation of latency data""" + return { + "latency_50th_ms": latency_50th_ms, + "latency_99th_ms": latency_99th_ms, + "latency_999th_ms": latency_999th_ms + } + + +def compute_aggregate_throughput(perf): + """Helper method for computing throughput after running a performance service.""" + aggregate_rate = sum([r['records_per_sec'] for r in perf.results]) + aggregate_mbps = sum([r['mbps'] for r in perf.results]) + + return throughput(aggregate_rate, aggregate_mbps) diff --git a/tests/kafkatest/tests/copycat_distributed_test.py b/tests/kafkatest/tests/copycat_distributed_test.py new file mode 100644 index 0000000000000..3afcd57d86d54 --- /dev/null +++ b/tests/kafkatest/tests/copycat_distributed_test.py @@ -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. + +from kafkatest.tests.kafka_test import KafkaTest +from kafkatest.services.copycat import CopycatDistributedService +from ducktape.utils.util import wait_until +import hashlib, subprocess, json, itertools + +class CopycatDistributedFileTest(KafkaTest): + """ + Simple test of Copycat in distributed mode, producing data from files on on Copycat cluster and consuming it on + another, validating the total output is identical to the input. + """ + + INPUT_FILE = "/mnt/copycat.input" + OUTPUT_FILE = "/mnt/copycat.output" + + TOPIC = "test" + OFFSETS_TOPIC = "copycat-offsets" + + FIRST_INPUT_LISTS = [["foo", "bar", "baz"], ["foo2", "bar2", "baz2"]] + FIRST_INPUTS = ["\n".join(input_list) + "\n" for input_list in FIRST_INPUT_LISTS] + SECOND_INPUT_LISTS = [["razz", "ma", "tazz"], ["razz2", "ma2", "tazz2"]] + SECOND_INPUTS = ["\n".join(input_list) + "\n" for input_list in SECOND_INPUT_LISTS] + + SCHEMA = { "type": "string", "optional": False } + + def __init__(self, test_context): + super(CopycatDistributedFileTest, self).__init__(test_context, num_zk=1, num_brokers=1, topics={ + 'test' : { 'partitions': 1, 'replication-factor': 1 } + }) + + self.source = CopycatDistributedService(test_context, 2, self.kafka, [self.INPUT_FILE]) + self.sink = CopycatDistributedService(test_context, 2, self.kafka, [self.OUTPUT_FILE]) + + def test_file_source_and_sink(self, converter="org.apache.kafka.copycat.json.JsonConverter", schemas=True): + assert converter != None, "converter type must be set" + # Template parameters + self.key_converter = converter + self.value_converter = converter + self.schemas = schemas + + # These need to be set + self.source.set_configs(self.render("copycat-distributed.properties"), self.render("copycat-file-source.properties")) + self.sink.set_configs(self.render("copycat-distributed.properties"), self.render("copycat-file-sink.properties")) + + self.source.start() + self.sink.start() + + # Generating data on the source node should generate new records and create new output on the sink node + for node, input in zip(self.source.nodes, self.FIRST_INPUTS): + node.account.ssh("echo -e -n " + repr(input) + " >> " + self.INPUT_FILE) + wait_until(lambda: self.validate_output(self.FIRST_INPUT_LISTS), timeout_sec=60, err_msg="Data added to input file was not seen in the output file in a reasonable amount of time.") + + # Restarting both should result in them picking up where they left off, + # only processing new data. + self.source.restart() + self.sink.restart() + + for node, input in zip(self.source.nodes, self.SECOND_INPUTS): + node.account.ssh("echo -e -n " + repr(input) + " >> " + self.INPUT_FILE) + wait_until(lambda: self.validate_output(self.FIRST_INPUT_LISTS + self.SECOND_INPUT_LISTS), timeout_sec=60, err_msg="Sink output file never converged to the same state as the input file") + + def validate_output(self, inputs): + try: + input_set = set(itertools.chain(*inputs)) + output_set = set(itertools.chain(*[ + [line.strip() for line in node.account.ssh_capture("cat " + self.OUTPUT_FILE)] for node in self.sink.nodes + ])) + return input_set == output_set + except subprocess.CalledProcessError: + return False diff --git a/tests/kafkatest/tests/copycat_test.py b/tests/kafkatest/tests/copycat_test.py new file mode 100644 index 0000000000000..1bd8ccbae696c --- /dev/null +++ b/tests/kafkatest/tests/copycat_test.py @@ -0,0 +1,94 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from kafkatest.tests.kafka_test import KafkaTest +from kafkatest.services.copycat import CopycatStandaloneService +from kafkatest.services.console_consumer import ConsoleConsumer +from ducktape.utils.util import wait_until +from ducktape.mark import parametrize +import hashlib, subprocess, json + +class CopycatStandaloneFileTest(KafkaTest): + """ + Simple test of Copycat that produces data from a file in one Copycat + standalone process and consumes it on another, validating the output is + identical to the input. + """ + + INPUT_FILE = "/mnt/copycat.input" + OUTPUT_FILE = "/mnt/copycat.output" + + OFFSETS_FILE = "/mnt/copycat.offsets" + + TOPIC = "test" + + FIRST_INPUT_LIST = ["foo", "bar", "baz"] + FIRST_INPUT = "\n".join(FIRST_INPUT_LIST) + "\n" + SECOND_INPUT_LIST = ["razz", "ma", "tazz"] + SECOND_INPUT = "\n".join(SECOND_INPUT_LIST) + "\n" + + SCHEMA = { "type": "string", "optional": False } + + def __init__(self, test_context): + super(CopycatStandaloneFileTest, self).__init__(test_context, num_zk=1, num_brokers=1, topics={ + 'test' : { 'partitions': 1, 'replication-factor': 1 } + }) + + self.source = CopycatStandaloneService(test_context, self.kafka, [self.INPUT_FILE, self.OFFSETS_FILE]) + self.sink = CopycatStandaloneService(test_context, self.kafka, [self.OUTPUT_FILE, self.OFFSETS_FILE]) + self.consumer_validator = ConsoleConsumer(test_context, 1, self.kafka, self.TOPIC, consumer_timeout_ms=1000) + + @parametrize(converter="org.apache.kafka.copycat.json.JsonConverter", schemas=True) + @parametrize(converter="org.apache.kafka.copycat.json.JsonConverter", schemas=False) + @parametrize(converter="org.apache.kafka.copycat.storage.StringConverter", schemas=None) + def test_file_source_and_sink(self, converter="org.apache.kafka.copycat.json.JsonConverter", schemas=True): + assert converter != None, "converter type must be set" + # Template parameters + self.key_converter = converter + self.value_converter = converter + self.schemas = schemas + + # These need to be set + self.source.set_configs(self.render("copycat-standalone.properties"), self.render("copycat-file-source.properties")) + self.sink.set_configs(self.render("copycat-standalone.properties"), self.render("copycat-file-sink.properties")) + + self.source.start() + self.sink.start() + + # Generating data on the source node should generate new records and create new output on the sink node + self.source.node.account.ssh("echo -e -n " + repr(self.FIRST_INPUT) + " >> " + self.INPUT_FILE) + wait_until(lambda: self.validate_output(self.FIRST_INPUT), timeout_sec=60, err_msg="Data added to input file was not seen in the output file in a reasonable amount of time.") + + # Restarting both should result in them picking up where they left off, + # only processing new data. + self.source.restart() + self.sink.restart() + + self.source.node.account.ssh("echo -e -n " + repr(self.SECOND_INPUT) + " >> " + self.INPUT_FILE) + wait_until(lambda: self.validate_output(self.FIRST_INPUT + self.SECOND_INPUT), timeout_sec=60, err_msg="Sink output file never converged to the same state as the input file") + + # Validate the format of the data in the Kafka topic + self.consumer_validator.run() + expected = json.dumps([line if not self.schemas else { "schema": self.SCHEMA, "payload": line } for line in self.FIRST_INPUT_LIST + self.SECOND_INPUT_LIST]) + decoder = (json.loads if converter.endswith("JsonConverter") else str) + actual = json.dumps([decoder(x) for x in self.consumer_validator.messages_consumed[1]]) + assert expected == actual, "Expected %s but saw %s in Kafka" % (expected, actual) + + def validate_output(self, value): + try: + output_hash = list(self.sink.node.account.ssh_capture("md5sum " + self.OUTPUT_FILE))[0].strip().split()[0] + return output_hash == hashlib.md5(value).hexdigest() + except subprocess.CalledProcessError: + return False diff --git a/tests/kafkatest/tests/kafka_test.py b/tests/kafkatest/tests/kafka_test.py new file mode 100644 index 0000000000000..7118721b5ded2 --- /dev/null +++ b/tests/kafkatest/tests/kafka_test.py @@ -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. + +from ducktape.tests.test import Test + + +from kafkatest.services.zookeeper import ZookeeperService +from kafkatest.services.kafka import KafkaService + + +class KafkaTest(Test): + """ + Helper class that manages setting up a Kafka cluster. Use this if the + default settings for Kafka are sufficient for your test; any customization + needs to be done manually. Your run() method should call tearDown and + setUp. The Zookeeper and Kafka services are available as the fields + KafkaTest.zk and KafkaTest.kafka. + """ + def __init__(self, test_context, num_zk, num_brokers, topics=None): + super(KafkaTest, self).__init__(test_context) + self.num_zk = num_zk + self.num_brokers = num_brokers + self.topics = topics + + self.zk = ZookeeperService(test_context, self.num_zk) + + self.kafka = KafkaService( + test_context, self.num_brokers, + self.zk, topics=self.topics) + + def setUp(self): + self.zk.start() + self.kafka.start() \ No newline at end of file diff --git a/tests/kafkatest/tests/log4j_appender_test.py b/tests/kafkatest/tests/log4j_appender_test.py new file mode 100644 index 0000000000000..0875dbe9e817a --- /dev/null +++ b/tests/kafkatest/tests/log4j_appender_test.py @@ -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. + + +from ducktape.utils.util import wait_until + +from kafkatest.tests.kafka_test import KafkaTest +from kafkatest.services.console_consumer import ConsoleConsumer +from kafkatest.services.kafka_log4j_appender import KafkaLog4jAppender + +import time + +TOPIC = "topic-log4j-appender" +MAX_MESSAGES = 100 + +class Log4jAppenderTest(KafkaTest): + """ + Tests KafkaLog4jAppender using VerifiableKafkaLog4jAppender that appends increasing ints to a Kafka topic + """ + def __init__(self, test_context): + super(Log4jAppenderTest, self).__init__(test_context, num_zk=1, num_brokers=1, topics={ + TOPIC: {'partitions': 1, 'replication-factor': 1} + }) + self.num_nodes = 1 + + self.appender = KafkaLog4jAppender(self.test_context, self.num_nodes, self.kafka, TOPIC, MAX_MESSAGES) + self.consumer = ConsoleConsumer(self.test_context, num_nodes=self.num_nodes, kafka=self.kafka, topic=TOPIC, consumer_timeout_ms=1000) + + def test_log4j_appender(self): + """ + Tests if KafkaLog4jAppender is producing to Kafka topic + :return: None + """ + self.appender.start() + self.appender.wait() + + t0 = time.time() + self.consumer.start() + node = self.consumer.nodes[0] + + wait_until(lambda: self.consumer.alive(node), + timeout_sec=10, backoff_sec=.2, err_msg="Consumer was too slow to start") + self.logger.info("consumer started in %s seconds " % str(time.time() - t0)) + + # Verify consumed messages count + expected_lines_count = MAX_MESSAGES * 2 # two times to account for new lines introduced by log4j + wait_until(lambda: len(self.consumer.messages_consumed[1]) == expected_lines_count, timeout_sec=10, + err_msg="Timed out waiting to consume expected number of messages.") + + self.consumer.stop_node(node) \ No newline at end of file diff --git a/tests/kafkatest/tests/replication_test.py b/tests/kafkatest/tests/replication_test.py new file mode 100644 index 0000000000000..d20cc220b4b32 --- /dev/null +++ b/tests/kafkatest/tests/replication_test.py @@ -0,0 +1,173 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from ducktape.tests.test import Test +from ducktape.utils.util import wait_until +from ducktape.mark import parametrize +from ducktape.mark import matrix + +from kafkatest.services.zookeeper import ZookeeperService +from kafkatest.services.kafka import KafkaService +from kafkatest.services.verifiable_producer import VerifiableProducer +from kafkatest.services.console_consumer import ConsoleConsumer, is_int + +import signal +import time + + +class ReplicationTest(Test): + """Replication tests. + These tests verify that replication provides simple durability guarantees by checking that data acked by + brokers is still available for consumption in the face of various failure scenarios.""" + + def __init__(self, test_context): + """:type test_context: ducktape.tests.test.TestContext""" + super(ReplicationTest, self).__init__(test_context=test_context) + + self.topic = "test_topic" + self.zk = ZookeeperService(test_context, num_nodes=1) + self.producer_throughput = 10000 + self.num_producers = 1 + self.num_consumers = 1 + + def setUp(self): + self.zk.start() + + def min_cluster_size(self): + """Override this since we're adding services outside of the constructor""" + return super(ReplicationTest, self).min_cluster_size() + self.num_producers + self.num_consumers + + def run_with_failure(self, failure, interbroker_security_protocol): + """This is the top-level test template. + + The steps are: + Produce messages in the background while driving some failure condition + When done driving failures, immediately stop producing + Consume all messages + Validate that messages acked by brokers were consumed + + Note that consuming is a bit tricky, at least with console consumer. The goal is to consume all messages + (foreach partition) in the topic. In this case, waiting for the last message may cause the consumer to stop + too soon since console consumer is consuming multiple partitions from a single thread and therefore we lose + ordering guarantees. + + Waiting on a count of consumed messages can be unreliable: if we stop consuming when num_consumed == num_acked, + we might exit early if some messages are duplicated (though not an issue here since producer retries==0) + + Therefore rely here on the consumer.timeout.ms setting which times out on the interval between successively + consumed messages. Since we run the producer to completion before running the consumer, this is a reliable + indicator that nothing is left to consume. + + """ + security_protocol='PLAINTEXT' + self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zk, + security_protocol=security_protocol, + interbroker_security_protocol=interbroker_security_protocol, + topics={self.topic: { + "partitions": 3, + "replication-factor": 3, + "min.insync.replicas": 2} + }) + self.kafka.start() + self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, self.topic, security_protocol=security_protocol, throughput=self.producer_throughput) + self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic, security_protocol=security_protocol, new_consumer=False, consumer_timeout_ms=3000, message_validator=is_int) + + # Produce in a background thread while driving broker failures + self.producer.start() + wait_until(lambda: self.producer.num_acked > 5, timeout_sec=5, + err_msg="Producer failed to start in a reasonable amount of time.") + failure() + self.producer.stop() + + self.acked = self.producer.acked + self.not_acked = self.producer.not_acked + self.logger.info("num not acked: %d" % self.producer.num_not_acked) + self.logger.info("num acked: %d" % self.producer.num_acked) + + # Consume all messages + self.consumer.start() + self.consumer.wait() + self.consumed = self.consumer.messages_consumed[1] + self.logger.info("num consumed: %d" % len(self.consumed)) + + # Check produced vs consumed + success, msg = self.validate() + + if not success: + self.mark_for_collect(self.producer) + + assert success, msg + + def clean_shutdown(self): + """Discover leader node for our topic and shut it down cleanly.""" + self.kafka.signal_leader(self.topic, partition=0, sig=signal.SIGTERM) + + def hard_shutdown(self): + """Discover leader node for our topic and shut it down with a hard kill.""" + self.kafka.signal_leader(self.topic, partition=0, sig=signal.SIGKILL) + + def clean_bounce(self): + """Chase the leader of one partition and restart it cleanly.""" + for i in range(5): + prev_leader_node = self.kafka.leader(topic=self.topic, partition=0) + self.kafka.restart_node(prev_leader_node, wait_sec=5, clean_shutdown=True) + + def hard_bounce(self): + """Chase the leader and restart it cleanly.""" + for i in range(5): + prev_leader_node = self.kafka.leader(topic=self.topic, partition=0) + self.kafka.restart_node(prev_leader_node, wait_sec=5, clean_shutdown=False) + + # Wait long enough for previous leader to probably be awake again + time.sleep(6) + + def validate(self): + """Check that produced messages were consumed.""" + + success = True + msg = "" + + if len(set(self.consumed)) != len(self.consumed): + # There are duplicates. This is ok, so report it but don't fail the test + msg += "There are duplicate messages in the log\n" + + if not set(self.consumed).issuperset(set(self.acked)): + # Every acked message must appear in the logs. I.e. consumed messages must be superset of acked messages. + acked_minus_consumed = set(self.producer.acked) - set(self.consumed) + success = False + msg += "At least one acked message did not appear in the consumed messages. acked_minus_consumed: " + str(acked_minus_consumed) + + if not success: + # Collect all the data logs if there was a failure + self.mark_for_collect(self.kafka) + + return success, msg + + + @matrix(interbroker_security_protocol=['PLAINTEXT', 'SSL']) + def test_clean_shutdown(self, interbroker_security_protocol): + self.run_with_failure(self.clean_shutdown, interbroker_security_protocol) + + @matrix(interbroker_security_protocol=['PLAINTEXT', 'SSL']) + def test_hard_shutdown(self, interbroker_security_protocol): + self.run_with_failure(self.hard_shutdown, interbroker_security_protocol) + + @matrix(interbroker_security_protocol=['PLAINTEXT', 'SSL']) + def test_clean_bounce(self, interbroker_security_protocol): + self.run_with_failure(self.clean_bounce, interbroker_security_protocol) + + @matrix(interbroker_security_protocol=['PLAINTEXT', 'SSL']) + def test_hard_bounce(self, interbroker_security_protocol): + self.run_with_failure(self.hard_bounce, interbroker_security_protocol) diff --git a/tests/kafkatest/tests/templates/copycat-distributed.properties b/tests/kafkatest/tests/templates/copycat-distributed.properties new file mode 100644 index 0000000000000..da47423f56201 --- /dev/null +++ b/tests/kafkatest/tests/templates/copycat-distributed.properties @@ -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. + +bootstrap.servers={{ kafka.bootstrap_servers() }} + +key.converter={{ key_converter|default("org.apache.kafka.copycat.json.JsonConverter") }} +value.converter={{ value_converter|default("org.apache.kafka.copycat.json.JsonConverter") }} +{% if key_converter is not defined or key_converter.endswith("JsonConverter") %} +key.converter.schemas.enable={{ schemas|default(True)|string|lower }} +{% endif %} +{% if value_converter is not defined or value_converter.endswith("JsonConverter") %} +value.converter.schemas.enable={{ schemas|default(True)|string|lower }} +{% endif %} + +offset.key.converter=org.apache.kafka.copycat.json.JsonConverter +offset.value.converter=org.apache.kafka.copycat.json.JsonConverter +offset.key.converter.schemas.enable=false +offset.value.converter.schemas.enable=false + +offset.storage.topic={{ OFFSETS_TOPIC }} diff --git a/tests/kafkatest/tests/templates/copycat-file-sink.properties b/tests/kafkatest/tests/templates/copycat-file-sink.properties new file mode 100644 index 0000000000000..77c43c7ffe287 --- /dev/null +++ b/tests/kafkatest/tests/templates/copycat-file-sink.properties @@ -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. + +name=local-file-sink +connector.class=org.apache.kafka.copycat.file.FileStreamSinkConnector +tasks.max=1 +file={{ OUTPUT_FILE }} +topics={{ TOPIC }} \ No newline at end of file diff --git a/tests/kafkatest/tests/templates/copycat-file-source.properties b/tests/kafkatest/tests/templates/copycat-file-source.properties new file mode 100644 index 0000000000000..68dabc21dcaaa --- /dev/null +++ b/tests/kafkatest/tests/templates/copycat-file-source.properties @@ -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. + +name=local-file-source +connector.class=org.apache.kafka.copycat.file.FileStreamSourceConnector +tasks.max=1 +file={{ INPUT_FILE }} +topic={{ TOPIC }} \ No newline at end of file diff --git a/tests/kafkatest/tests/templates/copycat-standalone.properties b/tests/kafkatest/tests/templates/copycat-standalone.properties new file mode 100644 index 0000000000000..39db6cec07e34 --- /dev/null +++ b/tests/kafkatest/tests/templates/copycat-standalone.properties @@ -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. + +bootstrap.servers={{ kafka.bootstrap_servers() }} + +key.converter={{ key_converter|default("org.apache.kafka.copycat.json.JsonConverter") }} +value.converter={{ value_converter|default("org.apache.kafka.copycat.json.JsonConverter") }} +{% if key_converter is not defined or key_converter.endswith("JsonConverter") %} +key.converter.schemas.enable={{ schemas|default(True)|string|lower }} +{% endif %} +{% if value_converter is not defined or value_converter.endswith("JsonConverter") %} +value.converter.schemas.enable={{ schemas|default(True)|string|lower }} +{% endif %} + +offset.key.converter=org.apache.kafka.copycat.json.JsonConverter +offset.value.converter=org.apache.kafka.copycat.json.JsonConverter +offset.key.converter.schemas.enable=false +offset.value.converter.schemas.enable=false + +offset.storage.file.filename={{ OFFSETS_FILE }} diff --git a/tests/kafkatest/utils/__init__.py b/tests/kafkatest/utils/__init__.py new file mode 100644 index 0000000000000..cff6d2b21125c --- /dev/null +++ b/tests/kafkatest/utils/__init__.py @@ -0,0 +1,15 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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 \ No newline at end of file diff --git a/tests/kafkatest/utils/remote_account.py b/tests/kafkatest/utils/remote_account.py new file mode 100644 index 0000000000000..b69a59180f613 --- /dev/null +++ b/tests/kafkatest/utils/remote_account.py @@ -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. + + +def file_exists(node, file): + """Quick and dirty check for existence of remote file.""" + try: + node.account.ssh("cat " + file, allow_fail=False) + return True + except: + return False + + +def line_count(node, file): + """Return the line count of file on node""" + out = [line for line in node.account.ssh_capture("wc -l %s" % file)] + if len(out) != 1: + raise Exception("Expected single line of output from wc -l") + + return int(out[0].strip().split(" ")[0]) \ No newline at end of file diff --git a/tests/kafkatest/utils/security_config.py b/tests/kafkatest/utils/security_config.py new file mode 100644 index 0000000000000..965f20922103b --- /dev/null +++ b/tests/kafkatest/utils/security_config.py @@ -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. + +import os +import subprocess + + +class Keytool(object): + + @staticmethod + def generate_keystore_truststore(ssl_dir='.'): + """ + Generate JKS keystore and truststore and return + Kafka SSL properties with these stores. + """ + ks_path = os.path.join(ssl_dir, 'test.keystore.jks') + ks_password = 'test-ks-passwd' + key_password = 'test-key-passwd' + ts_path = os.path.join(ssl_dir, 'test.truststore.jks') + ts_password = 'test-ts-passwd' + if os.path.exists(ks_path): + os.remove(ks_path) + if os.path.exists(ts_path): + os.remove(ts_path) + + Keytool.runcmd("keytool -genkeypair -alias test -keyalg RSA -keysize 2048 -keystore %s -storetype JKS -keypass %s -storepass %s -dname CN=systemtest" % (ks_path, key_password, ks_password)) + Keytool.runcmd("keytool -export -alias test -keystore %s -storepass %s -storetype JKS -rfc -file test.crt" % (ks_path, ks_password)) + Keytool.runcmd("keytool -import -alias test -file test.crt -keystore %s -storepass %s -storetype JKS -noprompt" % (ts_path, ts_password)) + os.remove('test.crt') + + return { + 'ssl.keystore.location' : ks_path, + 'ssl.keystore.password' : ks_password, + 'ssl.key.password' : key_password, + 'ssl.truststore.location' : ts_path, + 'ssl.truststore.password' : ts_password + } + + @staticmethod + def runcmd(cmd): + proc = subprocess.Popen(cmd, shell=True, stdout=subprocess.PIPE, stderr=subprocess.STDOUT) + proc.communicate() + if proc.returncode != 0: + raise subprocess.CalledProcessError(proc.returncode, cmd) + + +class SecurityConfig(object): + + PLAINTEXT = 'PLAINTEXT' + SSL = 'SSL' + SSL_DIR = "/mnt/ssl" + KEYSTORE_PATH = "/mnt/ssl/test.keystore.jks" + TRUSTSTORE_PATH = "/mnt/ssl/test.truststore.jks" + + ssl_stores = Keytool.generate_keystore_truststore('.') + + def __init__(self, security_protocol, template_props=""): + """ + Initialize the security properties for the node and copy + keystore and truststore to the remote node if the transport protocol + is SSL. If security_protocol is None, the protocol specified in the + template properties file is used. If no protocol is specified in the + template properties either, PLAINTEXT is used as default. + """ + + if security_protocol is None: + security_protocol = self.get_property('security.protocol', template_props) + if security_protocol is None: + security_protocol = SecurityConfig.PLAINTEXT + elif security_protocol not in [SecurityConfig.PLAINTEXT, SecurityConfig.SSL]: + raise Exception("Invalid security.protocol in template properties: " + security_protocol) + + self.properties = { + 'security.protocol' : security_protocol, + 'ssl.keystore.location' : SecurityConfig.KEYSTORE_PATH, + 'ssl.keystore.password' : SecurityConfig.ssl_stores['ssl.keystore.password'], + 'ssl.key.password' : SecurityConfig.ssl_stores['ssl.key.password'], + 'ssl.truststore.location' : SecurityConfig.TRUSTSTORE_PATH, + 'ssl.truststore.password' : SecurityConfig.ssl_stores['ssl.truststore.password'] + } + + def setup_node(self, node): + if self.security_protocol == SecurityConfig.SSL: + node.account.ssh("mkdir -p %s" % SecurityConfig.SSL_DIR, allow_fail=False) + node.account.scp_to(SecurityConfig.ssl_stores['ssl.keystore.location'], SecurityConfig.KEYSTORE_PATH) + node.account.scp_to(SecurityConfig.ssl_stores['ssl.truststore.location'], SecurityConfig.TRUSTSTORE_PATH) + + def clean_node(self, node): + if self.security_protocol == SecurityConfig.SSL: + node.account.ssh("rm -rf %s" % SecurityConfig.SSL_DIR, allow_fail=False) + + def get_property(self, prop_name, template_props=""): + """ + Get property value from the string representation of + a properties file. + """ + value = None + for line in template_props.split("\n"): + items = line.split("=") + if len(items) == 2 and items[0].strip() == prop_name: + value = str(items[1].strip()) + return value + + @property + def security_protocol(self): + return self.properties['security.protocol'] + + def __str__(self): + """ + Return properties as string with line separators. + This is used to append security config properties to + a properties file. + """ + + prop_str = "" + if self.security_protocol == SecurityConfig.SSL: + for key, value in self.properties.items(): + prop_str += ("\n" + key + "=" + value) + prop_str += "\n" + return prop_str + diff --git a/tests/setup.py b/tests/setup.py new file mode 100644 index 0000000000000..d637eb8dd0bdf --- /dev/null +++ b/tests/setup.py @@ -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. +# see kafka.server.KafkaConfig for additional details and defaults + +from setuptools import find_packages, setup + +setup(name="kafkatest", + version="0.9.0.dev0", + description="Apache Kafka System Tests", + author="Apache Kafka", + platforms=["any"], + license="apache2.0", + packages=find_packages(), + include_package_data=True, + install_requires=["ducktape==0.3.2"] + ) diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java b/tools/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java similarity index 81% rename from clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java rename to tools/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java index ac8615082e21c..1a9cf04cdccd7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java +++ b/tools/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java @@ -15,17 +15,10 @@ 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.ProducerRecord; -import org.apache.kafka.clients.producer.RecordMetadata; +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 < 4) { System.err.println("USAGE: java " + ProducerPerformance.class.getName() + @@ -46,33 +39,25 @@ public static void main(String[] args) throws Exception { throw new IllegalArgumentException("Invalid property: " + args[i]); props.put(pieces[0], pieces[1]); } - KafkaProducer producer = new KafkaProducer(props); + 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(topicName, payload); - long sleepTime = NS_PER_SEC / throughput; - long sleepDeficitNs = 0; + ProducerRecord record = new ProducerRecord(topicName, payload); Stats stats = new Stats(numRecords, 5000); + long startMs = System.currentTimeMillis(); + + ThroughputThrottler throttler = new ThroughputThrottler(throughput, startMs); for (int i = 0; i < numRecords; i++) { - long sendStart = System.currentTimeMillis(); - Callback cb = stats.nextCompletion(sendStart, payload.length, stats); + long sendStartMs = System.currentTimeMillis(); + Callback cb = stats.nextCompletion(sendStartMs, 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) { - sleepDeficitNs += sleepTime; - if (sleepDeficitNs >= MIN_SLEEP_NS) { - long sleepMs = sleepDeficitNs / 1000000; - long sleepNs = sleepDeficitNs - sleepMs * 1000000; - Thread.sleep(sleepMs, (int) sleepNs); - sleepDeficitNs = 0; - } + if (throttler.shouldThrottle(i, sendStartMs)) { + throttler.throttle(); } } @@ -163,9 +148,9 @@ public void newWindow() { } 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); + long elapsed = System.currentTimeMillis() - start; + double recsPerSec = 1000.0 * count / (double) elapsed; + double mbPerSec = 1000.0 * this.bytes / (double) elapsed / (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, diff --git a/tools/src/main/java/org/apache/kafka/clients/tools/ThroughputThrottler.java b/tools/src/main/java/org/apache/kafka/clients/tools/ThroughputThrottler.java new file mode 100644 index 0000000000000..06c443f576cf9 --- /dev/null +++ b/tools/src/main/java/org/apache/kafka/clients/tools/ThroughputThrottler.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.clients.tools; + + +/** + * This class helps producers throttle throughput. + * + * If targetThroughput >= 0, the resulting average throughput will be approximately + * min(targetThroughput, maximumPossibleThroughput). If targetThroughput < 0, + * no throttling will occur. + * + * To use, do this between successive send attempts: + *

                  + *     {@code     
                  + *      if (throttler.shouldThrottle(...)) {
                  + *          throttler.throttle();
                  + *      } 
                  + *     } 
                  + * 
                  + * + * Note that this can be used to throttle message throughput or data throughput. + */ +public class ThroughputThrottler { + + 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; + + long sleepTimeNs; + long sleepDeficitNs = 0; + long targetThroughput = -1; + long startMs; + + /** + * @param targetThroughput Can be messages/sec or bytes/sec + * @param startMs When the very first message is sent + */ + public ThroughputThrottler(long targetThroughput, long startMs) { + this.startMs = startMs; + this.targetThroughput = targetThroughput; + this.sleepTimeNs = targetThroughput > 0 ? + NS_PER_SEC / targetThroughput : + Long.MAX_VALUE; + } + + /** + * @param amountSoFar bytes produced so far if you want to throttle data throughput, or + * messages produced so far if you want to throttle message throughput. + * @param sendStartMs timestamp of the most recently sent message + * @return + */ + public boolean shouldThrottle(long amountSoFar, long sendStartMs) { + if (this.targetThroughput < 0) { + // No throttling in this case + return false; + } + + float elapsedMs = (sendStartMs - startMs) / 1000.f; + return elapsedMs > 0 && (amountSoFar / elapsedMs) > this.targetThroughput; + } + + /** + * Occasionally blocks for small amounts of time to achieve targetThroughput. + * + * Note that if targetThroughput is 0, this will block extremely aggressively. + */ + public void throttle() { + if (targetThroughput == 0) { + try { + Thread.sleep(Long.MAX_VALUE); + } catch (InterruptedException e) { + // do nothing + } + return; + } + + // throttle throughput by sleeping, on average, + // (1 / this.throughput) seconds between "things sent" + sleepDeficitNs += sleepTimeNs; + + // If enough sleep deficit has accumulated, sleep a little + if (sleepDeficitNs >= MIN_SLEEP_NS) { + long sleepMs = sleepDeficitNs / 1000000; + long sleepNs = sleepDeficitNs - sleepMs * 1000000; + + long sleepStartNs = System.nanoTime(); + try { + Thread.sleep(sleepMs, (int) sleepNs); + sleepDeficitNs = 0; + } catch (InterruptedException e) { + // If sleep is cut short, reduce deficit by the amount of + // time we actually spent sleeping + long sleepElapsedNs = System.nanoTime() - sleepStartNs; + if (sleepElapsedNs <= sleepDeficitNs) { + sleepDeficitNs -= sleepElapsedNs; + } + } + } + } +} + + \ No newline at end of file diff --git a/tools/src/main/java/org/apache/kafka/clients/tools/VerifiableLog4jAppender.java b/tools/src/main/java/org/apache/kafka/clients/tools/VerifiableLog4jAppender.java new file mode 100644 index 0000000000000..bf289b72954db --- /dev/null +++ b/tools/src/main/java/org/apache/kafka/clients/tools/VerifiableLog4jAppender.java @@ -0,0 +1,162 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 net.sourceforge.argparse4j.ArgumentParsers; +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.ArgumentParserException; +import net.sourceforge.argparse4j.inf.Namespace; +import org.apache.log4j.Logger; +import org.apache.log4j.PropertyConfigurator; + +import java.io.IOException; +import java.util.Properties; + +import static net.sourceforge.argparse4j.impl.Arguments.store; + +/** + * Primarily intended for use with system testing, this appender produces message + * to Kafka on each "append" request. For example, this helps with end-to-end tests + * of KafkaLog4jAppender. + * + * When used as a command-line tool, it appends increasing integers. It will produce a + * fixed number of messages unless the default max-messages -1 is used, in which case + * it appends indefinitely. + */ + +public class VerifiableLog4jAppender { + Logger logger = Logger.getLogger(VerifiableLog4jAppender.class); + + // If maxMessages < 0, log until the process is killed externally + private long maxMessages = -1; + + // Hook to trigger logging thread to stop logging messages + private volatile boolean stopLogging = false; + + /** Get the command-line argument parser. */ + private static ArgumentParser argParser() { + ArgumentParser parser = ArgumentParsers + .newArgumentParser("verifiable-log4j-appender") + .defaultHelp(true) + .description("This tool produces increasing integers to the specified topic using KafkaLog4jAppender."); + + parser.addArgument("--topic") + .action(store()) + .required(true) + .type(String.class) + .metavar("TOPIC") + .help("Produce messages to this topic."); + + parser.addArgument("--broker-list") + .action(store()) + .required(true) + .type(String.class) + .metavar("HOST1:PORT1[,HOST2:PORT2[...]]") + .dest("brokerList") + .help("Comma-separated list of Kafka brokers in the form HOST1:PORT1,HOST2:PORT2,..."); + + parser.addArgument("--max-messages") + .action(store()) + .required(false) + .setDefault(-1) + .type(Integer.class) + .metavar("MAX-MESSAGES") + .dest("maxMessages") + .help("Produce this many messages. If -1, produce messages until the process is killed externally."); + + parser.addArgument("--acks") + .action(store()) + .required(false) + .setDefault("-1") + .type(String.class) + .choices("0", "1", "-1") + .metavar("ACKS") + .help("Acks required on each produced message. See Kafka docs on request.required.acks for details."); + + return parser; + } + + /** Construct a VerifiableLog4jAppender object from command-line arguments. */ + public static VerifiableLog4jAppender createFromArgs(String[] args) { + ArgumentParser parser = argParser(); + VerifiableLog4jAppender producer = null; + + try { + Namespace res = parser.parseArgs(args); + + int maxMessages = res.getInt("maxMessages"); + String topic = res.getString("topic"); + + + Properties props = new Properties(); + props.setProperty("log4j.rootLogger", "INFO, KAFKA"); + props.setProperty("log4j.appender.KAFKA", "org.apache.kafka.log4jappender.KafkaLog4jAppender"); + props.setProperty("log4j.appender.KAFKA.layout", "org.apache.log4j.PatternLayout"); + props.setProperty("log4j.appender.KAFKA.layout.ConversionPattern", "%-5p: %c - %m%n"); + props.setProperty("log4j.appender.KAFKA.BrokerList", res.getString("brokerList")); + props.setProperty("log4j.appender.KAFKA.Topic", topic); + props.setProperty("log4j.appender.KAFKA.RequiredNumAcks", res.getString("acks")); + props.setProperty("log4j.appender.KAFKA.SyncSend", "true"); + props.setProperty("log4j.logger.kafka.log4j", "INFO, KAFKA"); + + producer = new VerifiableLog4jAppender(props, maxMessages); + } catch (ArgumentParserException e) { + if (args.length == 0) { + parser.printHelp(); + System.exit(0); + } else { + parser.handleError(e); + System.exit(1); + } + } + + return producer; + } + + + public VerifiableLog4jAppender(Properties props, int maxMessages) { + this.maxMessages = maxMessages; + PropertyConfigurator.configure(props); + } + + public static void main(String[] args) throws IOException { + + final VerifiableLog4jAppender appender = createFromArgs(args); + boolean infinite = appender.maxMessages < 0; + + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + // Trigger main thread to stop producing messages + appender.stopLogging = true; + } + }); + + long maxMessages = infinite ? Long.MAX_VALUE : appender.maxMessages; + for (long i = 0; i < maxMessages; i++) { + if (appender.stopLogging) { + break; + } + appender.append(String.format("%d", i)); + } + } + + private void append(String msg) { + logger.info(msg); + } +} diff --git a/tools/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java b/tools/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java new file mode 100644 index 0000000000000..a79f78e305c53 --- /dev/null +++ b/tools/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java @@ -0,0 +1,324 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +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.utils.Utils; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +import static net.sourceforge.argparse4j.impl.Arguments.store; + +import net.sourceforge.argparse4j.ArgumentParsers; +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.ArgumentParserException; +import net.sourceforge.argparse4j.inf.Namespace; + +/** + * Primarily intended for use with system testing, this producer prints metadata + * in the form of JSON to stdout on each "send" request. For example, this helps + * with end-to-end correctness tests by making externally visible which messages have been + * acked and which have not. + * + * When used as a command-line tool, it produces increasing integers. It will produce a + * fixed number of messages unless the default max-messages -1 is used, in which case + * it produces indefinitely. + * + * If logging is left enabled, log output on stdout can be easily ignored by checking + * whether a given line is valid JSON. + */ +public class VerifiableProducer { + + String topic; + private Producer producer; + // If maxMessages < 0, produce until the process is killed externally + private long maxMessages = -1; + + // Number of messages for which acks were received + private long numAcked = 0; + + // Number of send attempts + private long numSent = 0; + + // Throttle message throughput if this is set >= 0 + private long throughput; + + // Hook to trigger producing thread to stop sending messages + private boolean stopProducing = false; + + public VerifiableProducer( + Properties producerProps, String topic, int throughput, int maxMessages) { + + this.topic = topic; + this.throughput = throughput; + this.maxMessages = maxMessages; + this.producer = new KafkaProducer(producerProps); + } + + /** Get the command-line argument parser. */ + private static ArgumentParser argParser() { + ArgumentParser parser = ArgumentParsers + .newArgumentParser("verifiable-producer") + .defaultHelp(true) + .description("This tool produces increasing integers to the specified topic and prints JSON metadata to stdout on each \"send\" request, making externally visible which messages have been acked and which have not."); + + parser.addArgument("--topic") + .action(store()) + .required(true) + .type(String.class) + .metavar("TOPIC") + .help("Produce messages to this topic."); + + parser.addArgument("--broker-list") + .action(store()) + .required(true) + .type(String.class) + .metavar("HOST1:PORT1[,HOST2:PORT2[...]]") + .dest("brokerList") + .help("Comma-separated list of Kafka brokers in the form HOST1:PORT1,HOST2:PORT2,..."); + + parser.addArgument("--max-messages") + .action(store()) + .required(false) + .setDefault(-1) + .type(Integer.class) + .metavar("MAX-MESSAGES") + .dest("maxMessages") + .help("Produce this many messages. If -1, produce messages until the process is killed externally."); + + parser.addArgument("--throughput") + .action(store()) + .required(false) + .setDefault(-1) + .type(Integer.class) + .metavar("THROUGHPUT") + .help("If set >= 0, throttle maximum message throughput to *approximately* THROUGHPUT messages/sec."); + + parser.addArgument("--acks") + .action(store()) + .required(false) + .setDefault(-1) + .type(Integer.class) + .choices(0, 1, -1) + .metavar("ACKS") + .help("Acks required on each produced message. See Kafka docs on request.required.acks for details."); + + parser.addArgument("--producer.config") + .action(store()) + .required(false) + .type(String.class) + .metavar("CONFIG_FILE") + .help("Producer config properties file."); + + return parser; + } + + /** Construct a VerifiableProducer object from command-line arguments. */ + public static VerifiableProducer createFromArgs(String[] args) { + ArgumentParser parser = argParser(); + VerifiableProducer producer = null; + + try { + Namespace res; + res = parser.parseArgs(args); + + int maxMessages = res.getInt("maxMessages"); + String topic = res.getString("topic"); + int throughput = res.getInt("throughput"); + String configFile = res.getString("producer.config"); + + Properties producerProps = new Properties(); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, res.getString("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"); + producerProps.put(ProducerConfig.ACKS_CONFIG, Integer.toString(res.getInt("acks"))); + // No producer retries + producerProps.put("retries", "0"); + if (configFile != null) { + try { + producerProps.putAll(Utils.loadProps(configFile)); + } catch (IOException e) { + throw new ArgumentParserException(e.getMessage(), parser); + } + } + + producer = new VerifiableProducer(producerProps, topic, throughput, maxMessages); + } catch (ArgumentParserException e) { + if (args.length == 0) { + parser.printHelp(); + System.exit(0); + } else { + parser.handleError(e); + System.exit(1); + } + } + + return producer; + } + + /** Produce a message with given key and value. */ + public void send(String key, String value) { + ProducerRecord record = new ProducerRecord(topic, key, value); + numSent++; + try { + producer.send(record, new PrintInfoCallback(key, value)); + } catch (Exception e) { + + synchronized (System.out) { + System.out.println(errorString(e, key, value, System.currentTimeMillis())); + } + } + } + + /** Close the producer to flush any remaining messages. */ + public void close() { + producer.close(); + } + + /** + * Return JSON string encapsulating basic information about the exception, as well + * as the key and value which triggered the exception. + */ + String errorString(Exception e, String key, String value, Long nowMs) { + assert e != null : "Expected non-null exception."; + + Map errorData = new HashMap<>(); + errorData.put("class", this.getClass().toString()); + errorData.put("name", "producer_send_error"); + + errorData.put("time_ms", nowMs); + errorData.put("exception", e.getClass().toString()); + errorData.put("message", e.getMessage()); + errorData.put("topic", this.topic); + errorData.put("key", key); + errorData.put("value", value); + + return toJsonString(errorData); + } + + String successString(RecordMetadata recordMetadata, String key, String value, Long nowMs) { + assert recordMetadata != null : "Expected non-null recordMetadata object."; + + Map successData = new HashMap<>(); + successData.put("class", this.getClass().toString()); + successData.put("name", "producer_send_success"); + + successData.put("time_ms", nowMs); + successData.put("topic", this.topic); + successData.put("partition", recordMetadata.partition()); + successData.put("offset", recordMetadata.offset()); + successData.put("key", key); + successData.put("value", value); + + return toJsonString(successData); + } + + private String toJsonString(Map data) { + String json; + try { + ObjectMapper mapper = new ObjectMapper(); + json = mapper.writeValueAsString(data); + } catch (JsonProcessingException e) { + json = "Bad data can't be written as json: " + e.getMessage(); + } + return json; + } + + /** Callback which prints errors to stdout when the producer fails to send. */ + private class PrintInfoCallback implements Callback { + + private String key; + private String value; + + PrintInfoCallback(String key, String value) { + this.key = key; + this.value = value; + } + + public void onCompletion(RecordMetadata recordMetadata, Exception e) { + synchronized (System.out) { + if (e == null) { + VerifiableProducer.this.numAcked++; + System.out.println(successString(recordMetadata, this.key, this.value, System.currentTimeMillis())); + } else { + System.out.println(errorString(e, this.key, this.value, System.currentTimeMillis())); + } + } + } + } + + public static void main(String[] args) throws IOException { + + final VerifiableProducer producer = createFromArgs(args); + final long startMs = System.currentTimeMillis(); + boolean infinite = producer.maxMessages < 0; + + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + // Trigger main thread to stop producing messages + producer.stopProducing = true; + + // Flush any remaining messages + producer.close(); + + // Print a summary + long stopMs = System.currentTimeMillis(); + double avgThroughput = 1000 * ((producer.numAcked) / (double) (stopMs - startMs)); + + Map data = new HashMap<>(); + data.put("class", producer.getClass().toString()); + data.put("name", "tool_data"); + data.put("sent", producer.numSent); + data.put("acked", producer.numAcked); + data.put("target_throughput", producer.throughput); + data.put("avg_throughput", avgThroughput); + + System.out.println(producer.toJsonString(data)); + } + }); + + ThroughputThrottler throttler = new ThroughputThrottler(producer.throughput, startMs); + long maxMessages = infinite ? Long.MAX_VALUE : producer.maxMessages; + for (long i = 0; i < maxMessages; i++) { + if (producer.stopProducing) { + break; + } + long sendStartMs = System.currentTimeMillis(); + producer.send(null, String.format("%d", i)); + + if (throttler.shouldThrottle(i, sendStartMs)) { + throttler.throttle(); + } + } + } + +} diff --git a/vagrant/README.md b/vagrant/README.md new file mode 100644 index 0000000000000..6fa8e78a4dce6 --- /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 (1.5.0 or 1.4 currently required due to implementation changes in the plugin) + $ vagrant plugin install vagrant-hostmanager --plugin-version 1.5.0 + # 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/aws/aws-access-keys-commands b/vagrant/aws/aws-access-keys-commands new file mode 100644 index 0000000000000..9c923f86f3f8f --- /dev/null +++ b/vagrant/aws/aws-access-keys-commands @@ -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. +# see kafka.server.KafkaConfig for additional details and defaults + +if [ -z "$AWS_IAM" ];then + echo "Warning: AWS_IAM is not set" +fi + +export AWS_ACCESS_KEY=`curl -s http://169.254.169.254/latest/meta-data/iam/security-credentials/$AWS_IAM | grep AccessKeyId | awk -F\" '{ print $4 }'` +export AWS_SECRET_KEY=`curl -s http://169.254.169.254/latest/meta-data/iam/security-credentials/$AWS_IAM | grep SecretAccessKey | awk -F\" '{ print $4 }'` +export AWS_SESSION_TOKEN=`curl -s http://169.254.169.254/latest/meta-data/iam/security-credentials/$AWS_IAM | grep Token | awk -F\" '{ print $4 }'` + +if [ -z "$AWS_ACCESS_KEY" ]; then + echo "Failed to populate environment variables AWS_ACCESS_KEY, AWS_SECRET_KEY, and AWS_SESSION_TOKEN." + echo "AWS_IAM is currently $AWS_IAM. Double-check that this is correct. If not set, add this command to your .bashrc file:" + echo "export AWS_IAM= # put this into your ~/.bashrc" +fi diff --git a/vagrant/aws/aws-example-Vagrantfile.local b/vagrant/aws/aws-example-Vagrantfile.local new file mode 100644 index 0000000000000..c3b075b9420d4 --- /dev/null +++ b/vagrant/aws/aws-example-Vagrantfile.local @@ -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. +# see kafka.server.KafkaConfig for additional details and defaults + +# Use this template Vagrantfile.local for running system tests on aws +# To use it, move it to the base kafka directory and rename +# it to Vagrantfile.local, and adjust variables as needed. +ec3_instance_type = "m3.medium" +num_zookeepers = 0 +num_brokers = 0 +num_workers = 9 +ec2_keypair_name = kafkatest +ec2_keypair_file = ../kafkatest.pem +ec2_security_groups = ['kafkatest'] +ec2_region = 'us-west-2' +ec2_ami = "ami-29ebb519" diff --git a/vagrant/aws/aws-init.sh b/vagrant/aws/aws-init.sh new file mode 100755 index 0000000000000..61519280db3e2 --- /dev/null +++ b/vagrant/aws/aws-init.sh @@ -0,0 +1,73 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +#!/bin/bash + +# This script can be used to set up a driver machine on aws from which you will run tests +# or bring up your mini Kafka cluster. + +# Install dependencies +sudo apt-get install -y maven openjdk-6-jdk build-essential \ + ruby-dev zlib1g-dev realpath python-setuptools + +base_dir=`dirname $0`/../.. + +if [ -z `which vagrant` ]; then + echo "Installing vagrant..." + wget https://dl.bintray.com/mitchellh/vagrant/vagrant_1.7.2_x86_64.deb + sudo dpkg -i vagrant_1.7.2_x86_64.deb + rm -f vagrant_1.7.2_x86_64.deb +fi + +# Install necessary vagrant plugins +# Note: Do NOT install vagrant-cachier since it doesn't work on AWS and only +# adds log noise +vagrant_plugins="vagrant-aws vagrant-hostmanager" +existing=`vagrant plugin list` +for plugin in $vagrant_plugins; do + echo $existing | grep $plugin > /dev/null + if [ $? != 0 ]; then + vagrant plugin install $plugin + fi +done + +# Create Vagrantfile.local as a convenience +if [ ! -e "$base_dir/Vagrantfile.local" ]; then + cp $base_dir/aws/aws-example-Vagrantfile.local $base_dir/Vagrantfile.local +fi + +gradle="gradle-2.2.1" +if [ -z `which gradle` ] && [ ! -d $base_dir/$gradle ]; then + if [ ! -e $gradle-bin.zip ]; then + wget https://services.gradle.org/distributions/$gradle-bin.zip + fi + unzip $gradle-bin.zip + rm -rf $gradle-bin.zip + mv $gradle $base_dir/$gradle +fi + +# Ensure aws access keys are in the environment when we use a EC2 driver machine +LOCAL_HOSTNAME=$(hostname -d) +if [[ ${LOCAL_HOSTNAME} =~ .*\.compute\.internal ]]; then + grep "AWS ACCESS KEYS" ~/.bashrc > /dev/null + if [ $? != 0 ]; then + echo "# --- AWS ACCESS KEYS ---" >> ~/.bashrc + echo ". `realpath $base_dir/aws/aws-access-keys-commands`" >> ~/.bashrc + echo "# -----------------------" >> ~/.bashrc + source ~/.bashrc + fi +fi + diff --git a/vagrant/base.sh b/vagrant/base.sh new file mode 100644 index 0000000000000..133f10a95622c --- /dev/null +++ b/vagrant/base.sh @@ -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. + +#!/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 + +# For EC2 nodes, we want to use /mnt, which should have the local disk. On local +# VMs, we can just create it if it doesn't exist and use it like we'd use +# /tmp. Eventually, we'd like to also support more directories, e.g. when EC2 +# instances have multiple local disks. +if [ ! -e /mnt ]; then + mkdir /mnt +fi +chmod a+rwx /mnt 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/package-base-box.sh b/vagrant/package-base-box.sh new file mode 100755 index 0000000000000..5ac7f0e5356d8 --- /dev/null +++ b/vagrant/package-base-box.sh @@ -0,0 +1,75 @@ +#!/usr/bin/env 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. + +# This script automates the process of creating and packaging +# a new vagrant base_box. For use locally (not aws). + +base_dir=`dirname $0`/.. +cd $base_dir + +backup_vagrantfile=backup_Vagrantfile.local +local_vagrantfile=Vagrantfile.local + +# Restore original Vagrantfile.local, if it exists +function revert_vagrantfile { + rm -f $local_vagrantfile + if [ -e $backup_vagrantfile ]; then + mv $backup_vagrantfile $local_vagrantfile + fi +} + +function clean_up { + echo "Cleaning up..." + vagrant destroy -f + rm -f package.box + revert_vagrantfile +} + +# Name of the new base box +base_box="kafkatest-worker" + +# vagrant VM name +worker_name="worker1" + +echo "Destroying vagrant machines..." +vagrant destroy -f + +echo "Removing $base_box from vagrant..." +vagrant box remove $base_box + +echo "Bringing up a single vagrant machine from scratch..." +if [ -e $local_vagrantfile ]; then + mv $local_vagrantfile $backup_vagrantfile +fi +echo "num_workers = 1" > $local_vagrantfile +echo "num_brokers = 0" >> $local_vagrantfile +echo "num_zookeepers = 0" >> $local_vagrantfile +vagrant up +up_status=$? +if [ $up_status != 0 ]; then + echo "Failed to bring up a template vm, please try running again." + clean_up + exit $up_status +fi + +echo "Packaging $worker_name..." +vagrant package $worker_name + +echo "Adding new base box $base_box to vagrant..." +vagrant box add $base_box package.box + +clean_up + diff --git a/vagrant/system-test-Vagrantfile.local b/vagrant/system-test-Vagrantfile.local new file mode 100644 index 0000000000000..0ec04af06fb78 --- /dev/null +++ b/vagrant/system-test-Vagrantfile.local @@ -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. +# see kafka.server.KafkaConfig for additional details and defaults + +# Use this example Vagrantfile.local for running system tests +# To use it, move it to the base kafka directory and rename +# it to Vagrantfile.local +num_zookeepers = 0 +num_brokers = 0 +num_workers = 9 +base_box = "kafkatest-worker" 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 &